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

    -- * Usage Notes #usage_notes#

    -- ** Resource Management #resource_management#
    -- $resource_management

    -- ** Concurrency #concurrency#
    -- $concurrency

    -- ** ACID properties #acid#
    -- $acid

    -- ** Sharing #sharing#
    -- $sharing

    -- * Sessions #sessions#
    Session,
    withSession,
    openSession,
    closeSession,

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

    -- ** Table Lookups #table_lookups#
    member,
    members,
    lookup,
    lookups,
    rangeLookup,

    -- ** Table Updates #table_updates#
    insert,
    inserts,
    delete,
    deletes,
    update,
    updates,

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

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

    -- * 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
    ),
    MergePolicy (LazyLevelling),
    SizeRatio (Four),
    WriteBufferAlloc (AllocNumEntries),
    BloomFilterAlloc (AllocFixed, AllocRequestFPR),
    FencePointerIndexType (OrdinaryIndex, CompactIndex),
    DiskCachePolicy (..),
    MergeSchedule (..),

    -- ** 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,

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

import           Control.ActionRegistry (mapExceptionWithActionRegistry)
import           Control.Exception.Base (Exception, SomeException (..))
import           Data.Bifunctor (Bifunctor (..))
import           Data.Coerce (coerce)
import           Data.Kind (Type)
import           Data.List.NonEmpty (NonEmpty (..))
import           Data.Typeable (TypeRep)
import           Data.Vector (Vector)
import           Data.Void (Void)
import           Database.LSMTree (BloomFilterAlloc, CursorClosedError (..),
                     DiskCachePolicy, FencePointerIndexType,
                     InvalidSnapshotNameError (..), MergePolicy, MergeSchedule,
                     OverrideDiskCachePolicy (..), Range (..), RawBytes,
                     ResolveAsFirst (..), SerialiseKey (..),
                     SerialiseKeyOrderPreserving, SerialiseValue (..),
                     SessionClosedError (..), SizeRatio,
                     SnapshotCorruptedError (..),
                     SnapshotDoesNotExistError (..), SnapshotExistsError (..),
                     SnapshotLabel (..), SnapshotName,
                     SnapshotNotCompatibleError (..), TableClosedError (..),
                     TableConfig (..), TableCorruptedError (..),
                     TableTooLargeError (..), UnionCredits (..), UnionDebt (..),
                     WriteBufferAlloc, isValidSnapshotName, packSlice,
                     serialiseKeyIdentity, serialiseKeyIdentityUpToSlicing,
                     serialiseKeyMinimalSize, serialiseKeyPreservesOrdering,
                     serialiseValueIdentity, serialiseValueIdentityUpToSlicing,
                     toSnapshotName)
import qualified Database.LSMTree as LSMT
import           Prelude hiding (lookup, take, takeWhile)

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

{- $setup
>>> import Prelude hiding (lookup)
>>> import Data.ByteString.Short (ShortByteString)
>>> import Data.String (IsString)
>>> import Data.Word (Word64)
>>> import System.Directory (createDirectoryIfMissing, getTemporaryDirectory)
>>> import System.FilePath ((</>))

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

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

>>> :{
runExample :: (Session -> Table Key Value -> IO a) -> IO a
runExample action = do
  tmpDir <- getTemporaryDirectory
  let sessionDir = tmpDir </> "doctest_Database_LSMTree_Simple"
  createDirectoryIfMissing True sessionDir
  withSession sessionDir $ \session ->
    withTable session $ \table ->
      action session table
:}
-}

{- $example

>>> :{
runExample $ \session table -> do
  insert table 0 "Hello"
  insert table 1 "World"
  lookup table 0
:}
Just (Value "Hello")
-}

--------------------------------------------------------------------------------
-- Resource Management
--------------------------------------------------------------------------------

{- $resource_management
This package uses explicit resource management. The 'Session', 'Table', and 'Cursor'
handles hold open resources, such as file handles, which must be explicitly released.
Every operation that allocates a resource is paired with another operation to releases
that resource. For each pair of allocate and release operations there is a bracketed
function that combines the two.

+------------+--------------------------+-------------------------+-------------------+
| Resource   | Bracketed #bracketed#    | Allocate #allocate#     | Release #release# |
+============+==========================+=========================+===================+
| 'Session'  | 'withSession'            | 'openSession'           | 'closeSession'    |
+------------+--------------------------+-------------------------+-------------------+
| 'Table'    | 'withTable'              | 'newTable'              | 'closeTable'      |
+            +--------------------------+-------------------------+                   +
|            | 'withDuplicate'          | 'duplicate'             |                   |
+            +--------------------------+-------------------------+                   +
|            | 'withUnion'              | 'union'                 |                   |
+            +--------------------------+-------------------------+                   +
|            | 'withIncrementalUnion'   | 'incrementalUnion'      |                   |
+            +--------------------------+-------------------------+                   +
|            | 'withTableFromSnapshot'  | 'openTableFromSnapshot' |                   |
+------------+--------------------------+-------------------------+-------------------+
| 'Cursor'   | 'withCursor'             | 'newCursor'             | 'closeCursor'     |
+------------+--------------------------+-------------------------+-------------------+

To prevent resource and memory leaks due to asynchronous exceptions,
it is recommended to use the [bracketed](#bracketed) functions whenever
possible, and otherwise:

*   Run functions that allocate and release a resource with asynchronous
    exceptions masked.
*   Ensure that every use allocate operation is followed by the corresponding release
    operation even in the presence of asynchronous exceptions, e.g., using 'bracket'.
-}

--------------------------------------------------------------------------------
-- Concurrency
--------------------------------------------------------------------------------

{- $concurrency
Table handles may be used concurrently from multiple Haskell threads,
and doing read operations concurrently may result in improved throughput,
as it can take advantage of CPU and I\/O parallelism. However, concurrent
use of write operations may introduces races. Specifically:

* It is a race to read and write the same table concurrently.
* It is a race to write and write the same table concurrently.
* It is /not/ a race to read and read the same table concurrently.
* It is /not/ a race to read or write /separate/ tables concurrently.

For the purposes of the above rules:

* The read operations are 'lookup', 'rangeLookup', 'duplicate', `union`, 'saveSnapshot', 'newCursor', and their variants.
* The write operations are 'insert', 'delete', 'update', 'closeTable', and their variants.

It is possible to read from a stable view of a table while concurrently writing to
the table by using 'duplicate' and performing the read operations on the duplicate.
However, this requires that the 'duplicate' operation /happens before/ the subsequent
writes, as it is a race to duplicate concurrently with any writes.
As this package does not provide any construct for synchronisation or atomic
operations, this ordering of operations must be accomplished by the user through
other means.

A 'Cursor' creates a stable view of a table and can safely be read while
modifying the original table. However, reading the 'next' key\/value pair from
a cursor locks the view, so concurrent reads on the same cursor block.
This is because 'next' updates the cursor's current position.

Session handles may be used concurrently from multiple Haskell threads,
but concurrent use of read and write operations may introduce races.
Specifically, it is a race to use `listSnapshots` and `deleteSnapshots`
with the same session handle concurrently.
-}

--------------------------------------------------------------------------------
-- ACID properties
--------------------------------------------------------------------------------

{- $acid
This text copies liberally from https://en.wikipedia.org/wiki/ACID and related wiki pages.

Atomicity, consistency, isolation, and durability (ACID) are important
properties of database transactions.
They guarantee data validity despite errors, power failures, and other mishaps.
A /transaction/ is a sequence of database operations that satisfy the ACID properties.

@lsm-tree@ does not support transactions in the typical sense that many relational databases do,
where transactions can be built from smaller components/actions,
e.g., reads and writes of individual cells.
Instead, the public API only exposes functions that individually form a transaction;
there are no smaller building blocks.
An example of such a transaction is 'updates'.

An @lsm-tree@ transaction still perform multiple database actions /internally/,
but transactions themselves are not composable into larger transactions,
so it should be expected that table contents can change between transactions in a concurrent setting.
A consistent view of a table can be created,
so that independent transactions have access to their own version of the database state (see [concurrency](#g:concurreny)).

All @lsm-tree@ transactions are designed for atomicity, consistency, and isolation (ACI),
assuming that users of the library perform proper [resource management](#g:resource-management).
Durability is only guaranteed when saving a [snapshot](#g:snapshots),
which is the only method of stopping and restarting tables.

We currently cannot guarantee consistency in the presence of synchronous and asynchronous exceptions,
eventhough major strides were made to make it so.
The safest course of action when an internal exception is encountered is to stop and restart:
close the session along with all its tables and cursors, reopen the session,
and load a previous saved table snapshot.
-}

--------------------------------------------------------------------------------
-- Sharing
--------------------------------------------------------------------------------

{- $sharing
Tables created via 'duplicate' or 'union' will initially share as much of their
in-memory and on-disk data as possible with the tables they were created from.
Over time as these related tables are modified, the contents of the tables will
diverge, which means that the tables will share less and less.

Sharing of in-memory data is not preserved by snapshots, but sharing of on-disk
data is partially preserved.
Existing files for runs are shared, but files for ongoing merges are not.
Opening a table from a snapshot (using 'openTableFromSnapshot' or
'withTableFromSnapshot') is expensive, but creating a snapshot (using
'saveSnapshot') is relatively cheap.
-}

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

{- |
A session stores context that is shared by multiple tables.

Each session is associated with one session directory where the files
containing table data are stored. Each session locks its session directory.
There can only be one active session for each session directory at a time.
If a database is must be accessed from multiple parts of a program,
one session should be opened and shared between those parts of the program.
Session directories cannot be shared between OS processes.
-}
type Session :: Type
newtype Session = Session (LSMT.Session IO)

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

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.
-}
withSession ::
    forall a.
    -- | The session directory.
    FilePath ->
    (Session -> IO a) ->
    IO a
withSession :: forall a. FilePath -> (Session -> IO a) -> IO a
withSession FilePath
dir Session -> IO a
action = do
    let tracer :: Tracer IO LSMTreeTrace
tracer = Tracer IO LSMTreeTrace
forall a. Monoid a => a
mempty
    FilePath -> IO a -> IO a
forall a. FilePath -> IO a -> IO a
_convertSessionDirErrors FilePath
dir (IO a -> IO a) -> IO a -> IO a
forall a b. (a -> b) -> a -> b
$
        Tracer IO LSMTreeTrace -> FilePath -> (Session IO -> IO a) -> IO a
forall a.
Tracer IO LSMTreeTrace -> FilePath -> (Session IO -> IO a) -> IO a
LSMT.withSessionIO Tracer IO LSMTreeTrace
tracer FilePath
dir (Session -> IO a
action (Session -> IO a) -> (Session IO -> Session) -> Session IO -> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Session IO -> Session
Session)

{- |
Open a session from a session directory.

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.
-}
openSession ::
    -- | The session directory.
    FilePath ->
    IO Session
openSession :: FilePath -> IO Session
openSession FilePath
dir = do
    let tracer :: Tracer IO LSMTreeTrace
tracer = Tracer IO LSMTreeTrace
forall a. Monoid a => a
mempty
    FilePath -> IO Session -> IO Session
forall a. FilePath -> IO a -> IO a
_convertSessionDirErrors FilePath
dir (IO Session -> IO Session) -> IO Session -> IO Session
forall a b. (a -> b) -> a -> b
$ do
        Session IO -> Session
Session (Session IO -> Session) -> IO (Session IO) -> IO Session
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Tracer IO LSMTreeTrace -> FilePath -> IO (Session IO)
LSMT.openSessionIO Tracer IO LSMTreeTrace
tracer FilePath
dir

{- |
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.
-}
closeSession ::
    Session ->
    IO ()
closeSession :: Session -> IO ()
closeSession (Session Session IO
session) =
    Session IO -> IO ()
forall (m :: * -> *). IOLike m => Session m -> m ()
LSMT.closeSession Session IO
session

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

{- |
A table is a handle to an individual LSM-tree key\/value store with both in-memory and on-disk parts.

__Warning:__ Tables are ephemeral. Once you close a table, its data is lost forever. To persist tables, use [snapshots](#g:snapshots).
-}
type role Table nominal nominal

type Table :: Type -> Type -> Type
newtype Table k v = Table (LSMT.Table IO k (LSMT.ResolveAsFirst v) Void)

{- |
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.
-}
withTable ::
    forall k v a.
    Session ->
    (Table k v -> IO a) ->
    IO a
withTable :: forall k v a. Session -> (Table k v -> IO a) -> IO a
withTable (Session Session IO
session) Table k v -> IO a
action =
    Session IO -> (Table IO k (ResolveAsFirst v) Void -> IO a) -> IO a
forall (m :: * -> *) k v b a.
IOLike m =>
Session m -> (Table m k v b -> m a) -> m a
LSMT.withTable Session IO
session (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

-- | Variant of 'withTable' that accepts [table configuration](#g:table_configuration).
withTableWith ::
    forall k v a.
    TableConfig ->
    Session ->
    (Table k v -> IO a) ->
    IO a
withTableWith :: forall k v a. TableConfig -> Session -> (Table k v -> IO a) -> IO a
withTableWith TableConfig
tableConfig (Session Session IO
session) Table k v -> IO a
action =
    TableConfig
-> Session IO
-> (Table IO k (ResolveAsFirst v) Void -> IO a)
-> IO a
forall (m :: * -> *) k v b a.
IOLike m =>
TableConfig -> Session m -> (Table m k v b -> m a) -> m a
LSMT.withTableWith TableConfig
tableConfig Session IO
session (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
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.
-}
newTable ::
    forall k v.
    Session ->
    IO (Table k v)
newTable :: forall k v. Session -> IO (Table k v)
newTable (Session Session IO
session) =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Session IO -> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
IOLike m =>
Session m -> m (Table m k v b)
LSMT.newTable Session IO
session

{- |
Variant of 'newTable' that accepts [table configuration](#g:table_configuration).
-}
newTableWith ::
    forall k v.
    TableConfig ->
    Session ->
    IO (Table k v)
newTableWith :: forall k v. TableConfig -> Session -> IO (Table k v)
newTableWith TableConfig
tableConfig (Session Session IO
session) =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> TableConfig
-> Session IO -> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
IOLike m =>
TableConfig -> Session m -> m (Table m k v b)
LSMT.newTableWith TableConfig
tableConfig Session IO
session

{- |
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).
-}
closeTable ::
    forall k v.
    Table k v ->
    IO ()
closeTable :: forall k v. Table k v -> IO ()
closeTable (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> IO ()
forall (m :: * -> *) k v b. IOLike m => Table m k v b -> m ()
LSMT.closeTable Table IO k (ResolveAsFirst v) Void
table

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

{- |
Check if the key is a member of the 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})\).

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.
-}
member ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    k ->
    IO Bool
member :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> k -> IO Bool
member (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> k -> IO Bool
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> k -> m Bool
LSMT.member Table IO k (ResolveAsFirst v) Void
table

{- |
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
-}
members ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    Vector k ->
    IO (Vector Bool)
members :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> Vector k -> IO (Vector Bool)
members (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> Vector k -> IO (Vector Bool)
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v) =>
Table m k v b -> Vector k -> m (Vector Bool)
LSMT.members Table IO k (ResolveAsFirst v) Void
table

-- | Internal helper. Get the value from a 'LSMT.LookupResult' from the full API.
getValue :: LSMT.LookupResult (ResolveAsFirst v) (LSMT.BlobRef IO Void) -> Maybe v
getValue :: forall v.
LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v
getValue =
    (ResolveAsFirst v -> v) -> Maybe (ResolveAsFirst v) -> Maybe v
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ResolveAsFirst v -> v
forall v. ResolveAsFirst v -> v
LSMT.unResolveAsFirst (Maybe (ResolveAsFirst v) -> Maybe v)
-> (LookupResult (ResolveAsFirst v) (BlobRef IO Void)
    -> Maybe (ResolveAsFirst v))
-> LookupResult (ResolveAsFirst v) (BlobRef IO Void)
-> Maybe v
forall b c a. (b -> c) -> (a -> b) -> a -> c
. LookupResult (ResolveAsFirst v) (BlobRef IO Void)
-> Maybe (ResolveAsFirst v)
forall v b. LookupResult v b -> Maybe v
LSMT.getValue

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

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.
-}
lookup ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    k ->
    IO (Maybe v)
lookup :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> k -> IO (Maybe v)
lookup (Table Table IO k (ResolveAsFirst v) Void
table) =
    (LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v)
-> IO (LookupResult (ResolveAsFirst v) (BlobRef IO Void))
-> IO (Maybe v)
forall a b. (a -> b) -> IO a -> IO b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v
forall v.
LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v
getValue (IO (LookupResult (ResolveAsFirst v) (BlobRef IO Void))
 -> IO (Maybe v))
-> (k -> IO (LookupResult (ResolveAsFirst v) (BlobRef IO Void)))
-> k
-> IO (Maybe v)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void
-> k -> IO (LookupResult (ResolveAsFirst v) (BlobRef IO Void))
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))
LSMT.lookup Table IO k (ResolveAsFirst v) Void
table

{- |
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
-}
lookups ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    Vector k ->
    IO (Vector (Maybe v))
lookups :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> Vector k -> IO (Vector (Maybe v))
lookups (Table Table IO k (ResolveAsFirst v) Void
table) =
    (Vector (LookupResult (ResolveAsFirst v) (BlobRef IO Void))
 -> Vector (Maybe v))
-> IO (Vector (LookupResult (ResolveAsFirst v) (BlobRef IO Void)))
-> IO (Vector (Maybe v))
forall a b. (a -> b) -> IO a -> IO b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v)
-> Vector (LookupResult (ResolveAsFirst v) (BlobRef IO Void))
-> Vector (Maybe v)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v
forall v.
LookupResult (ResolveAsFirst v) (BlobRef IO Void) -> Maybe v
getValue) (IO (Vector (LookupResult (ResolveAsFirst v) (BlobRef IO Void)))
 -> IO (Vector (Maybe v)))
-> (Vector k
    -> IO (Vector (LookupResult (ResolveAsFirst v) (BlobRef IO Void))))
-> Vector k
-> IO (Vector (Maybe v))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void
-> Vector k
-> IO (Vector (LookupResult (ResolveAsFirst v) (BlobRef IO Void)))
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)))
LSMT.lookups Table IO k (ResolveAsFirst v) Void
table

-- | Internal helper. Get a key\/value pair from an 'LSMT.Entry' from the full API.
getKeyValue :: LSMT.Entry k (ResolveAsFirst v) (LSMT.BlobRef IO Void) -> (k, v)
getKeyValue :: forall k v. Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
getKeyValue (LSMT.Entry k
k ResolveAsFirst v
v)             = (k
k, ResolveAsFirst v -> v
forall v. ResolveAsFirst v -> v
LSMT.unResolveAsFirst ResolveAsFirst v
v)
getKeyValue (LSMT.EntryWithBlob k
k ResolveAsFirst v
v !BlobRef IO Void
_b) = (k
k, ResolveAsFirst v -> v
forall v. ResolveAsFirst v -> v
LSMT.unResolveAsFirst ResolveAsFirst v
v)

{- |
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.
-}
rangeLookup ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    Range k ->
    IO (Vector (k, v))
rangeLookup :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> Range k -> IO (Vector (k, v))
rangeLookup (Table Table IO k (ResolveAsFirst v) Void
table) =
    (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))
 -> Vector (k, v))
-> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
-> IO (Vector (k, v))
forall a b. (a -> b) -> IO a -> IO b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v))
-> Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))
-> Vector (k, v)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
forall k v. Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
getKeyValue) (IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
 -> IO (Vector (k, v)))
-> (Range k
    -> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))))
-> Range k
-> IO (Vector (k, v))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void
-> Range k
-> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
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)))
LSMT.rangeLookup Table IO k (ResolveAsFirst v) Void
table

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

{- |
Insert a new key and value in the table.
If the key is already present in the table, the associated value is replaced with the given value.

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.
-}
insert ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    k ->
    v ->
    IO ()
insert :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> k -> v -> IO ()
insert (Table Table IO k (ResolveAsFirst v) Void
table) k
k v
v =
    Table IO k (ResolveAsFirst v) Void
-> k -> ResolveAsFirst v -> Maybe Void -> IO ()
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 ()
LSMT.insert Table IO k (ResolveAsFirst v) Void
table k
k (v -> ResolveAsFirst v
forall v. v -> ResolveAsFirst v
LSMT.ResolveAsFirst v
v) Maybe Void
forall a. Maybe a
Nothing

{- |
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
-}
inserts ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    Vector (k, v) ->
    IO ()
inserts :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> Vector (k, v) -> IO ()
inserts (Table Table IO k (ResolveAsFirst v) Void
table) Vector (k, v)
entries =
    Table IO k (ResolveAsFirst v) Void
-> Vector (k, ResolveAsFirst v, Maybe Void) -> IO ()
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 ()
LSMT.inserts Table IO k (ResolveAsFirst v) Void
table (((k, v) -> (k, ResolveAsFirst v, Maybe Void))
-> Vector (k, v) -> Vector (k, ResolveAsFirst v, Maybe Void)
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) -> (k
k, v -> ResolveAsFirst v
forall v. v -> ResolveAsFirst v
LSMT.ResolveAsFirst v
v, Maybe Void
forall a. Maybe a
Nothing)) Vector (k, v)
entries)

{- |
Delete a key and its value from the table.
If the key is not present in the table, the table is left unchanged.

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.
-}
delete ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    k ->
    IO ()
delete :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> k -> IO ()
delete (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> k -> IO ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> k -> m ()
LSMT.delete Table IO k (ResolveAsFirst v) Void
table

{- |
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
-}
deletes ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    Vector k ->
    IO ()
deletes :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> Vector k -> IO ()
deletes (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> Vector k -> IO ()
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, SerialiseValue v, ResolveValue v,
 SerialiseValue b) =>
Table m k v b -> Vector k -> m ()
LSMT.deletes Table IO k (ResolveAsFirst v) Void
table

-- | Internal helper. Convert from @'Maybe' v@ to an 'LSMT.Update'.
maybeValueToUpdate :: Maybe v -> LSMT.Update (ResolveAsFirst v) Void
maybeValueToUpdate :: forall v. Maybe v -> Update (ResolveAsFirst v) Void
maybeValueToUpdate =
    Update (ResolveAsFirst v) Void
-> (v -> Update (ResolveAsFirst v) Void)
-> Maybe v
-> Update (ResolveAsFirst v) Void
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Update (ResolveAsFirst v) Void
forall v b. Update v b
LSMT.Delete (\v
v -> ResolveAsFirst v -> Maybe Void -> Update (ResolveAsFirst v) Void
forall v b. v -> Maybe b -> Update v b
LSMT.Insert (v -> ResolveAsFirst v
forall v. v -> ResolveAsFirst v
LSMT.ResolveAsFirst v
v) Maybe Void
forall a. Maybe a
Nothing)

{- |
Update the value at a specific key:

* If the given value is 'Just', this operation acts as 'insert'.
* If the given value is 'Nothing', this operation acts as 'delete'.

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.
-}
update ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    k ->
    Maybe v ->
    IO ()
update :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> k -> Maybe v -> IO ()
update (Table Table IO k (ResolveAsFirst v) Void
table) k
k =
    Table IO k (ResolveAsFirst v) Void
-> k -> Update (ResolveAsFirst v) Void -> IO ()
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 ()
LSMT.update Table IO k (ResolveAsFirst v) Void
table k
k (Update (ResolveAsFirst v) Void -> IO ())
-> (Maybe v -> Update (ResolveAsFirst v) Void) -> Maybe v -> IO ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Maybe v -> Update (ResolveAsFirst v) Void
forall v. Maybe v -> Update (ResolveAsFirst v) Void
maybeValueToUpdate

{- |
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
-}
updates ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Table k v ->
    Vector (k, Maybe v) ->
    IO ()
updates :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Table k v -> Vector (k, Maybe v) -> IO ()
updates (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void
-> Vector (k, Update (ResolveAsFirst v) Void) -> IO ()
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 ()
LSMT.updates Table IO k (ResolveAsFirst v) Void
table (Vector (k, Update (ResolveAsFirst v) Void) -> IO ())
-> (Vector (k, Maybe v)
    -> Vector (k, Update (ResolveAsFirst v) Void))
-> Vector (k, Maybe v)
-> IO ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((k, Maybe v) -> (k, Update (ResolveAsFirst v) Void))
-> Vector (k, Maybe v)
-> Vector (k, Update (ResolveAsFirst v) Void)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((Maybe v -> Update (ResolveAsFirst v) Void)
-> (k, Maybe v) -> (k, Update (ResolveAsFirst v) Void)
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 Maybe v -> Update (ResolveAsFirst v) Void
forall v. Maybe v -> Update (ResolveAsFirst v) Void
maybeValueToUpdate)

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

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

The duplicate is an independent copy of the given table.
The duplicate is unaffected by subsequent updates to the given table and vice versa.

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.
-}
withDuplicate ::
    forall k v a.
    Table k v ->
    (Table k v -> IO a) ->
    IO a
withDuplicate :: forall k v a. Table k v -> (Table k v -> IO a) -> IO a
withDuplicate (Table Table IO k (ResolveAsFirst v) Void
table) Table k v -> IO a
action =
    Table IO k (ResolveAsFirst v) Void
-> (Table IO k (ResolveAsFirst v) Void -> IO a) -> IO a
forall (m :: * -> *) k v b a.
IOLike m =>
Table m k v b -> (Table m k v b -> m a) -> m a
LSMT.withDuplicate Table IO k (ResolveAsFirst v) Void
table (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

{- |
Duplicate a table.

The duplicate is an independent copy of the given table.
The duplicate is unaffected by subsequent updates to the given table and vice versa.

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.
-}
duplicate ::
    forall k v.
    Table k v ->
    IO (Table k v)
duplicate :: forall k v. Table k v -> IO (Table k v)
duplicate (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Table IO k (ResolveAsFirst v) Void
-> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m (Table m k v b)
LSMT.duplicate Table IO k (ResolveAsFirst v) Void
table

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

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

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'.
-}
withUnion ::
    forall k v a.
    Table k v ->
    Table k v ->
    (Table k v -> IO a) ->
    IO a
withUnion :: forall k v a. Table k v -> Table k v -> (Table k v -> IO a) -> IO a
withUnion (Table Table IO k (ResolveAsFirst v) Void
table1) (Table Table IO k (ResolveAsFirst v) Void
table2) Table k v -> IO a
action =
    Table IO k (ResolveAsFirst v) Void
-> Table IO k (ResolveAsFirst v) Void
-> (Table IO k (ResolveAsFirst v) Void -> IO a)
-> IO a
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
LSMT.withUnion Table IO k (ResolveAsFirst v) Void
table1 Table IO k (ResolveAsFirst v) Void
table2 (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

{- |
Variant of 'withUnions' that takes any number of tables.
-}
withUnions ::
    forall k v a.
    NonEmpty (Table k v) ->
    (Table k v -> IO a) ->
    IO a
withUnions :: forall k v a. NonEmpty (Table k v) -> (Table k v -> IO a) -> IO a
withUnions NonEmpty (Table k v)
tables Table k v -> IO a
action =
    NonEmpty (Table IO k (ResolveAsFirst v) Void)
-> (Table IO k (ResolveAsFirst v) Void -> IO a) -> IO a
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
LSMT.withUnions (NonEmpty (Table k v)
-> NonEmpty (Table IO k (ResolveAsFirst v) Void)
forall a b. Coercible a b => a -> b
coerce NonEmpty (Table k v)
tables) (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

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

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'.
-}
union ::
    forall k v.
    Table k v ->
    Table k v ->
    IO (Table k v)
union :: forall k v. Table k v -> Table k v -> IO (Table k v)
union (Table Table IO k (ResolveAsFirst v) Void
table1) (Table Table IO k (ResolveAsFirst v) Void
table2) =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Table IO k (ResolveAsFirst v) Void
-> Table IO k (ResolveAsFirst v) Void
-> IO (Table IO k (ResolveAsFirst v) Void)
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)
LSMT.union Table IO k (ResolveAsFirst v) Void
table1 Table IO k (ResolveAsFirst v) Void
table2

{- |
Variant of 'union' that takes any number of tables.
-}
unions ::
    forall k v.
    NonEmpty (Table k v) ->
    IO (Table k v)
unions :: forall k v. NonEmpty (Table k v) -> IO (Table k v)
unions NonEmpty (Table k v)
tables =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> NonEmpty (Table IO k (ResolveAsFirst v) Void)
-> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
NonEmpty (Table m k v b) -> m (Table m k v b)
LSMT.unions (NonEmpty (Table k v)
-> NonEmpty (Table IO k (ResolveAsFirst v) Void)
forall a b. Coercible a b => a -> b
coerce NonEmpty (Table k v)
tables)

{- |
Run an action with access to a table that incrementally computes the union of the given 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})\).

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'.
-}
withIncrementalUnion ::
    forall k v a.
    Table k v ->
    Table k v ->
    (Table k v -> IO a) ->
    IO a
withIncrementalUnion :: forall k v a. Table k v -> Table k v -> (Table k v -> IO a) -> IO a
withIncrementalUnion (Table Table IO k (ResolveAsFirst v) Void
table1) (Table Table IO k (ResolveAsFirst v) Void
table2) Table k v -> IO a
action =
    Table IO k (ResolveAsFirst v) Void
-> Table IO k (ResolveAsFirst v) Void
-> (Table IO k (ResolveAsFirst v) Void -> IO a)
-> IO a
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
LSMT.withIncrementalUnion Table IO k (ResolveAsFirst v) Void
table1 Table IO k (ResolveAsFirst v) Void
table2 (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

{- |
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.
-}
withIncrementalUnions ::
    forall k v a.
    NonEmpty (Table k v) ->
    (Table k v -> IO a) ->
    IO a
withIncrementalUnions :: forall k v a. NonEmpty (Table k v) -> (Table k v -> IO a) -> IO a
withIncrementalUnions NonEmpty (Table k v)
tables Table k v -> IO a
action =
    NonEmpty (Table IO k (ResolveAsFirst v) Void)
-> (Table IO k (ResolveAsFirst v) Void -> IO a) -> IO a
forall (m :: * -> *) k v b a.
IOLike m =>
NonEmpty (Table m k v b) -> (Table m k v b -> m a) -> m a
LSMT.withIncrementalUnions (NonEmpty (Table k v)
-> NonEmpty (Table IO k (ResolveAsFirst v) Void)
forall a b. Coercible a b => a -> b
coerce NonEmpty (Table k v)
tables) (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

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

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'.
-}
incrementalUnion ::
    forall k v.
    Table k v ->
    Table k v ->
    IO (Table k v)
incrementalUnion :: forall k v. Table k v -> Table k v -> IO (Table k v)
incrementalUnion (Table Table IO k (ResolveAsFirst v) Void
table1) (Table Table IO k (ResolveAsFirst v) Void
table2) = do
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Table IO k (ResolveAsFirst v) Void
-> Table IO k (ResolveAsFirst v) Void
-> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> Table m k v b -> m (Table m k v b)
LSMT.incrementalUnion Table IO k (ResolveAsFirst v) Void
table1 Table IO k (ResolveAsFirst v) Void
table2

{- |
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.
-}
incrementalUnions ::
    forall k v.
    NonEmpty (Table k v) ->
    IO (Table k v)
incrementalUnions :: forall k v. NonEmpty (Table k v) -> IO (Table k v)
incrementalUnions NonEmpty (Table k v)
tables = do
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> NonEmpty (Table IO k (ResolveAsFirst v) Void)
-> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
NonEmpty (Table m k v b) -> m (Table m k v b)
LSMT.unions (NonEmpty (Table k v)
-> NonEmpty (Table IO k (ResolveAsFirst v) Void)
forall a b. Coercible a b => a -> b
coerce NonEmpty (Table k v)
tables)

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

The worst-case disk I\/O complexity of this operation is \(O(0)\).
-}
remainingUnionDebt ::
    forall k v.
    Table k v ->
    IO UnionDebt
remainingUnionDebt :: forall k v. Table k v -> IO UnionDebt
remainingUnionDebt (Table Table IO k (ResolveAsFirst v) Void
table) =
    Table IO k (ResolveAsFirst v) Void -> IO UnionDebt
forall (m :: * -> *) k v b.
IOLike m =>
Table m k v b -> m UnionDebt
LSMT.remainingUnionDebt Table IO k (ResolveAsFirst v) Void
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 see what the current debt is.

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.

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.
-}
supplyUnionCredits ::
    forall k v.
    Table k v ->
    UnionCredits ->
    IO UnionCredits
supplyUnionCredits :: forall k v. Table k v -> UnionCredits -> IO UnionCredits
supplyUnionCredits (Table Table IO k (ResolveAsFirst v) Void
table) UnionCredits
credits =
    Table IO k (ResolveAsFirst v) Void
-> UnionCredits -> IO UnionCredits
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> UnionCredits -> m UnionCredits
LSMT.supplyUnionCredits Table IO k (ResolveAsFirst v) Void
table UnionCredits
credits

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

{- |
A cursor is a stable read-only iterator for a table.

A cursor iterates over the entries in a table following the order of the
serialised keys. After the cursor is created, updates to the referenced table
do not affect the cursor.

The name of this type references [database cursors](https://en.wikipedia.org/wiki/Cursor_(databases\)), not, e.g., text editor cursors.
-}
type role Cursor nominal nominal

type Cursor :: Type -> Type -> Type
newtype Cursor k v = Cursor (LSMT.Cursor IO k (ResolveAsFirst v) Void)

{- |
Run an action with access to 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})\).

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.
-}
withCursor ::
    forall k v a.
    Table k v ->
    (Cursor k v -> IO a) ->
    IO a
withCursor :: forall k v a. Table k v -> (Cursor k v -> IO a) -> IO a
withCursor (Table Table IO k (ResolveAsFirst v) Void
table) Cursor k v -> IO a
action =
    Table IO k (ResolveAsFirst v) Void
-> (Cursor IO k (ResolveAsFirst v) Void -> IO a) -> IO a
forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
Table m k v b -> (Cursor m k v b -> m a) -> m a
LSMT.withCursor Table IO k (ResolveAsFirst v) Void
table (Cursor k v -> IO a
action (Cursor k v -> IO a)
-> (Cursor IO k (ResolveAsFirst v) Void -> Cursor k v)
-> Cursor IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
forall k v. Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
Cursor)

{- |
Variant of 'withCursor' that starts at a given key.
-}
withCursorAtOffset ::
    forall k v a.
    (SerialiseKey k) =>
    Table k v ->
    k ->
    (Cursor k v -> IO a) ->
    IO a
withCursorAtOffset :: forall k v a.
SerialiseKey k =>
Table k v -> k -> (Cursor k v -> IO a) -> IO a
withCursorAtOffset (Table Table IO k (ResolveAsFirst v) Void
table) k
offsetKey Cursor k v -> IO a
action =
    Table IO k (ResolveAsFirst v) Void
-> k -> (Cursor IO k (ResolveAsFirst v) Void -> IO a) -> IO a
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
LSMT.withCursorAtOffset Table IO k (ResolveAsFirst v) Void
table k
offsetKey (Cursor k v -> IO a
action (Cursor k v -> IO a)
-> (Cursor IO k (ResolveAsFirst v) Void -> Cursor k v)
-> Cursor IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
forall k v. Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
Cursor)

{- |
Create a cursor for the given 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})\).

__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.
-}
newCursor ::
    forall k v.
    Table k v ->
    IO (Cursor k v)
newCursor :: forall k v. Table k v -> IO (Cursor k v)
newCursor (Table Table IO k (ResolveAsFirst v) Void
table) =
    Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
forall k v. Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
Cursor (Cursor IO k (ResolveAsFirst v) Void -> Cursor k v)
-> IO (Cursor IO k (ResolveAsFirst v) Void) -> IO (Cursor k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Table IO k (ResolveAsFirst v) Void
-> IO (Cursor IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Table m k v b -> m (Cursor m k v b)
LSMT.newCursor Table IO k (ResolveAsFirst v) Void
table

{- |
Variant of 'newCursor' that starts at a given key.
-}
newCursorAtOffset ::
    forall k v.
    (SerialiseKey k) =>
    Table k v ->
    k ->
    IO (Cursor k v)
newCursorAtOffset :: forall k v. SerialiseKey k => Table k v -> k -> IO (Cursor k v)
newCursorAtOffset (Table Table IO k (ResolveAsFirst v) Void
table) k
offsetKey =
    Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
forall k v. Cursor IO k (ResolveAsFirst v) Void -> Cursor k v
Cursor (Cursor IO k (ResolveAsFirst v) Void -> Cursor k v)
-> IO (Cursor IO k (ResolveAsFirst v) Void) -> IO (Cursor k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Table IO k (ResolveAsFirst v) Void
-> k -> IO (Cursor IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
(IOLike m, SerialiseKey k, ResolveValue v) =>
Table m k v b -> k -> m (Cursor m k v b)
LSMT.newCursorAtOffset Table IO k (ResolveAsFirst v) Void
table k
offsetKey

{- |
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.
-}
closeCursor ::
    forall k v.
    Cursor k v ->
    IO ()
closeCursor :: forall k v. Cursor k v -> IO ()
closeCursor (Cursor Cursor IO k (ResolveAsFirst v) Void
cursor) =
    Cursor IO k (ResolveAsFirst v) Void -> IO ()
forall (m :: * -> *) k v b. IOLike m => Cursor m k v b -> m ()
LSMT.closeCursor Cursor IO k (ResolveAsFirst v) Void
cursor

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

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.
-}
next ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Cursor k v ->
    IO (Maybe (k, v))
next :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Cursor k v -> IO (Maybe (k, v))
next (Cursor Cursor IO k (ResolveAsFirst v) Void
cursor) =
    (Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v))
-> Maybe (Entry k (ResolveAsFirst v) (BlobRef IO Void))
-> Maybe (k, v)
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
forall k v. Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
getKeyValue (Maybe (Entry k (ResolveAsFirst v) (BlobRef IO Void))
 -> Maybe (k, v))
-> IO (Maybe (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
-> IO (Maybe (k, v))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Cursor IO k (ResolveAsFirst v) Void
-> IO (Maybe (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
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)))
LSMT.next Cursor IO k (ResolveAsFirst v) Void
cursor

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

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.
-}
take ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Int ->
    Cursor k v ->
    IO (Vector (k, v))
take :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Int -> Cursor k v -> IO (Vector (k, v))
take Int
n (Cursor Cursor IO k (ResolveAsFirst v) Void
cursor) =
    (Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v))
-> Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))
-> Vector (k, v)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
forall k v. Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
getKeyValue (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))
 -> Vector (k, v))
-> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
-> IO (Vector (k, v))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int
-> Cursor IO k (ResolveAsFirst v) Void
-> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
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)))
LSMT.take Int
n Cursor IO k (ResolveAsFirst v) Void
cursor

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

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.
-}
takeWhile ::
    forall k v.
    (SerialiseKey k, SerialiseValue v) =>
    Int ->
    (k -> Bool) ->
    Cursor k v ->
    IO (Vector (k, v))
takeWhile :: forall k v.
(SerialiseKey k, SerialiseValue v) =>
Int -> (k -> Bool) -> Cursor k v -> IO (Vector (k, v))
takeWhile Int
n k -> Bool
p (Cursor Cursor IO k (ResolveAsFirst v) Void
cursor) =
    (Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v))
-> Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))
-> Vector (k, v)
forall a b. (a -> b) -> Vector a -> Vector b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
forall k v. Entry k (ResolveAsFirst v) (BlobRef IO Void) -> (k, v)
getKeyValue (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void))
 -> Vector (k, v))
-> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
-> IO (Vector (k, v))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Int
-> (k -> Bool)
-> Cursor IO k (ResolveAsFirst v) Void
-> IO (Vector (Entry k (ResolveAsFirst v) (BlobRef IO Void)))
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)))
LSMT.takeWhile Int
n k -> Bool
p Cursor IO k (ResolveAsFirst v) Void
cursor

--------------------------------------------------------------------------------
-- 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.

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.
-}
saveSnapshot ::
    forall k v.
    SnapshotName ->
    SnapshotLabel ->
    Table k v ->
    IO ()
saveSnapshot :: forall k v. SnapshotName -> SnapshotLabel -> Table k v -> IO ()
saveSnapshot SnapshotName
snapName SnapshotLabel
snapLabel (Table Table IO k (ResolveAsFirst v) Void
table) =
    SnapshotName
-> SnapshotLabel -> Table IO k (ResolveAsFirst v) Void -> IO ()
forall (m :: * -> *) k v b.
IOLike m =>
SnapshotName -> SnapshotLabel -> Table m k v b -> m ()
LSMT.saveSnapshot SnapshotName
snapName SnapshotLabel
snapLabel Table IO k (ResolveAsFirst v) Void
table

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

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.
-}
withTableFromSnapshot ::
    forall k v a.
    Session ->
    SnapshotName ->
    SnapshotLabel ->
    (Table k v -> IO a) ->
    IO a
withTableFromSnapshot :: forall k v a.
Session
-> SnapshotName -> SnapshotLabel -> (Table k v -> IO a) -> IO a
withTableFromSnapshot (Session Session IO
session) SnapshotName
snapName SnapshotLabel
snapLabel Table k v -> IO a
action =
    Session IO
-> SnapshotName
-> SnapshotLabel
-> (Table IO k (ResolveAsFirst v) Void -> IO a)
-> IO a
forall (m :: * -> *) k v b a.
(IOLike m, ResolveValue v) =>
Session m
-> SnapshotName -> SnapshotLabel -> (Table m k v b -> m a) -> m a
LSMT.withTableFromSnapshot Session IO
session SnapshotName
snapName SnapshotLabel
snapLabel (Table k v -> IO a
action (Table k v -> IO a)
-> (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> Table IO k (ResolveAsFirst v) Void
-> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table)

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

{- |
Open a table from a named snapshot.

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.
-}
openTableFromSnapshot ::
    forall k v.
    Session ->
    SnapshotName ->
    SnapshotLabel ->
    IO (Table k v)
openTableFromSnapshot :: forall k v.
Session -> SnapshotName -> SnapshotLabel -> IO (Table k v)
openTableFromSnapshot (Session Session IO
session) SnapshotName
snapName SnapshotLabel
snapLabel =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
LSMT.openTableFromSnapshot @IO @k @(ResolveAsFirst v) Session IO
session SnapshotName
snapName SnapshotLabel
snapLabel

{- |
Variant of 'openTableFromSnapshot' that accepts [table configuration overrides](#g:table_configuration_overrides).
-}
openTableFromSnapshotWith ::
    forall k v.
    OverrideDiskCachePolicy ->
    Session ->
    SnapshotName ->
    SnapshotLabel ->
    IO (Table k v)
openTableFromSnapshotWith :: forall k v.
OverrideDiskCachePolicy
-> Session -> SnapshotName -> SnapshotLabel -> IO (Table k v)
openTableFromSnapshotWith OverrideDiskCachePolicy
tableConfigOverride (Session Session IO
session) SnapshotName
snapName SnapshotLabel
snapLabel =
    Table IO k (ResolveAsFirst v) Void -> Table k v
forall k v. Table IO k (ResolveAsFirst v) Void -> Table k v
Table (Table IO k (ResolveAsFirst v) Void -> Table k v)
-> IO (Table IO k (ResolveAsFirst v) Void) -> IO (Table k v)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> OverrideDiskCachePolicy
-> Session IO
-> SnapshotName
-> SnapshotLabel
-> IO (Table IO k (ResolveAsFirst v) Void)
forall (m :: * -> *) k v b.
(IOLike m, ResolveValue v) =>
OverrideDiskCachePolicy
-> Session m -> SnapshotName -> SnapshotLabel -> m (Table m k v b)
LSMT.openTableFromSnapshotWith OverrideDiskCachePolicy
tableConfigOverride Session IO
session SnapshotName
snapName SnapshotLabel
snapLabel

{- |
Delete the named snapshot.

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.
-}
deleteSnapshot ::
    Session ->
    SnapshotName ->
    IO ()
deleteSnapshot :: Session -> SnapshotName -> IO ()
deleteSnapshot (Session Session IO
session) =
    Session IO -> SnapshotName -> IO ()
forall (m :: * -> *). IOLike m => Session m -> SnapshotName -> m ()
LSMT.deleteSnapshot Session IO
session

{- |
Check if the named snapshot exists.

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

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
-}
doesSnapshotExist ::
    Session ->
    SnapshotName ->
    IO Bool
doesSnapshotExist :: Session -> SnapshotName -> IO Bool
doesSnapshotExist (Session Session IO
session) =
    Session IO -> SnapshotName -> IO Bool
forall (m :: * -> *).
IOLike m =>
Session m -> SnapshotName -> m Bool
LSMT.doesSnapshotExist Session IO
session

{- |
List the names of all snapshots.

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

Throws the following exceptions:

['SessionClosedError']:
    If the session is closed.
-}
listSnapshots ::
    Session ->
    IO [SnapshotName]
listSnapshots :: Session -> IO [SnapshotName]
listSnapshots (Session Session IO
session) =
    Session IO -> IO [SnapshotName]
forall (m :: * -> *). IOLike m => Session m -> m [SnapshotName]
LSMT.listSnapshots Session IO
session

--------------------------------------------------------------------------------
-- Errors
--------------------------------------------------------------------------------

-- | The session directory does not exist.
data SessionDirDoesNotExistError
    = ErrSessionDirDoesNotExist !FilePath
    deriving stock (Int -> SessionDirDoesNotExistError -> ShowS
[SessionDirDoesNotExistError] -> ShowS
SessionDirDoesNotExistError -> FilePath
(Int -> SessionDirDoesNotExistError -> ShowS)
-> (SessionDirDoesNotExistError -> FilePath)
-> ([SessionDirDoesNotExistError] -> ShowS)
-> Show SessionDirDoesNotExistError
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> SessionDirDoesNotExistError -> ShowS
showsPrec :: Int -> SessionDirDoesNotExistError -> ShowS
$cshow :: SessionDirDoesNotExistError -> FilePath
show :: SessionDirDoesNotExistError -> FilePath
$cshowList :: [SessionDirDoesNotExistError] -> ShowS
showList :: [SessionDirDoesNotExistError] -> ShowS
Show, SessionDirDoesNotExistError -> SessionDirDoesNotExistError -> Bool
(SessionDirDoesNotExistError
 -> SessionDirDoesNotExistError -> Bool)
-> (SessionDirDoesNotExistError
    -> SessionDirDoesNotExistError -> Bool)
-> Eq SessionDirDoesNotExistError
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: SessionDirDoesNotExistError -> SessionDirDoesNotExistError -> Bool
== :: SessionDirDoesNotExistError -> SessionDirDoesNotExistError -> Bool
$c/= :: SessionDirDoesNotExistError -> SessionDirDoesNotExistError -> Bool
/= :: SessionDirDoesNotExistError -> SessionDirDoesNotExistError -> Bool
Eq)
    deriving anyclass (Show SessionDirDoesNotExistError
Typeable SessionDirDoesNotExistError
(Typeable SessionDirDoesNotExistError,
 Show SessionDirDoesNotExistError) =>
(SessionDirDoesNotExistError -> SomeException)
-> (SomeException -> Maybe SessionDirDoesNotExistError)
-> (SessionDirDoesNotExistError -> FilePath)
-> Exception SessionDirDoesNotExistError
SomeException -> Maybe SessionDirDoesNotExistError
SessionDirDoesNotExistError -> FilePath
SessionDirDoesNotExistError -> SomeException
forall e.
(Typeable e, Show e) =>
(e -> SomeException)
-> (SomeException -> Maybe e) -> (e -> FilePath) -> Exception e
$ctoException :: SessionDirDoesNotExistError -> SomeException
toException :: SessionDirDoesNotExistError -> SomeException
$cfromException :: SomeException -> Maybe SessionDirDoesNotExistError
fromException :: SomeException -> Maybe SessionDirDoesNotExistError
$cdisplayException :: SessionDirDoesNotExistError -> FilePath
displayException :: SessionDirDoesNotExistError -> FilePath
Exception)

-- | The session directory is locked by another active session.
data SessionDirLockedError
    = ErrSessionDirLocked !FilePath
    deriving stock (Int -> SessionDirLockedError -> ShowS
[SessionDirLockedError] -> ShowS
SessionDirLockedError -> FilePath
(Int -> SessionDirLockedError -> ShowS)
-> (SessionDirLockedError -> FilePath)
-> ([SessionDirLockedError] -> ShowS)
-> Show SessionDirLockedError
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> SessionDirLockedError -> ShowS
showsPrec :: Int -> SessionDirLockedError -> ShowS
$cshow :: SessionDirLockedError -> FilePath
show :: SessionDirLockedError -> FilePath
$cshowList :: [SessionDirLockedError] -> ShowS
showList :: [SessionDirLockedError] -> ShowS
Show, SessionDirLockedError -> SessionDirLockedError -> Bool
(SessionDirLockedError -> SessionDirLockedError -> Bool)
-> (SessionDirLockedError -> SessionDirLockedError -> Bool)
-> Eq SessionDirLockedError
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: SessionDirLockedError -> SessionDirLockedError -> Bool
== :: SessionDirLockedError -> SessionDirLockedError -> Bool
$c/= :: SessionDirLockedError -> SessionDirLockedError -> Bool
/= :: SessionDirLockedError -> SessionDirLockedError -> Bool
Eq)
    deriving anyclass (Show SessionDirLockedError
Typeable SessionDirLockedError
(Typeable SessionDirLockedError, Show SessionDirLockedError) =>
(SessionDirLockedError -> SomeException)
-> (SomeException -> Maybe SessionDirLockedError)
-> (SessionDirLockedError -> FilePath)
-> Exception SessionDirLockedError
SomeException -> Maybe SessionDirLockedError
SessionDirLockedError -> FilePath
SessionDirLockedError -> SomeException
forall e.
(Typeable e, Show e) =>
(e -> SomeException)
-> (SomeException -> Maybe e) -> (e -> FilePath) -> Exception e
$ctoException :: SessionDirLockedError -> SomeException
toException :: SessionDirLockedError -> SomeException
$cfromException :: SomeException -> Maybe SessionDirLockedError
fromException :: SomeException -> Maybe SessionDirLockedError
$cdisplayException :: SessionDirLockedError -> FilePath
displayException :: SessionDirLockedError -> FilePath
Exception)

-- | The session directory is corrupted, e.g., it misses required files or contains unexpected files.
data SessionDirCorruptedError
    = ErrSessionDirCorrupted !FilePath
    deriving stock (Int -> SessionDirCorruptedError -> ShowS
[SessionDirCorruptedError] -> ShowS
SessionDirCorruptedError -> FilePath
(Int -> SessionDirCorruptedError -> ShowS)
-> (SessionDirCorruptedError -> FilePath)
-> ([SessionDirCorruptedError] -> ShowS)
-> Show SessionDirCorruptedError
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> SessionDirCorruptedError -> ShowS
showsPrec :: Int -> SessionDirCorruptedError -> ShowS
$cshow :: SessionDirCorruptedError -> FilePath
show :: SessionDirCorruptedError -> FilePath
$cshowList :: [SessionDirCorruptedError] -> ShowS
showList :: [SessionDirCorruptedError] -> ShowS
Show, SessionDirCorruptedError -> SessionDirCorruptedError -> Bool
(SessionDirCorruptedError -> SessionDirCorruptedError -> Bool)
-> (SessionDirCorruptedError -> SessionDirCorruptedError -> Bool)
-> Eq SessionDirCorruptedError
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: SessionDirCorruptedError -> SessionDirCorruptedError -> Bool
== :: SessionDirCorruptedError -> SessionDirCorruptedError -> Bool
$c/= :: SessionDirCorruptedError -> SessionDirCorruptedError -> Bool
/= :: SessionDirCorruptedError -> SessionDirCorruptedError -> Bool
Eq)
    deriving anyclass (Show SessionDirCorruptedError
Typeable SessionDirCorruptedError
(Typeable SessionDirCorruptedError,
 Show SessionDirCorruptedError) =>
(SessionDirCorruptedError -> SomeException)
-> (SomeException -> Maybe SessionDirCorruptedError)
-> (SessionDirCorruptedError -> FilePath)
-> Exception SessionDirCorruptedError
SomeException -> Maybe SessionDirCorruptedError
SessionDirCorruptedError -> FilePath
SessionDirCorruptedError -> SomeException
forall e.
(Typeable e, Show e) =>
(e -> SomeException)
-> (SomeException -> Maybe e) -> (e -> FilePath) -> Exception e
$ctoException :: SessionDirCorruptedError -> SomeException
toException :: SessionDirCorruptedError -> SomeException
$cfromException :: SomeException -> Maybe SessionDirCorruptedError
fromException :: SomeException -> Maybe SessionDirCorruptedError
$cdisplayException :: SessionDirCorruptedError -> FilePath
displayException :: SessionDirCorruptedError -> FilePath
Exception)

{- | Internal helper. Convert:

*   t'LSMT.SessionDirDoesNotExistError' to t'SessionDirDoesNotExistError';
*   t'LSMT.SessionDirLockedError'       to t'SessionDirLockedError'; and
*   t'LSMT.SessionDirCorruptedError'    to t'SessionDirCorruptedError'.
-}
_convertSessionDirErrors ::
    forall a.
    FilePath ->
    IO a ->
    IO a
_convertSessionDirErrors :: forall a. FilePath -> IO a -> IO a
_convertSessionDirErrors FilePath
sessionDir =
    (SessionDirDoesNotExistError -> SomeException) -> IO a -> IO a
forall e1 e2 (m :: * -> *) a.
(Exception e1, Exception e2, MonadCatch m) =>
(e1 -> e2) -> m a -> m a
mapExceptionWithActionRegistry (\(LSMT.ErrSessionDirDoesNotExist FsErrorPath
_fsErrorPath) -> SessionDirDoesNotExistError -> SomeException
forall e. Exception e => e -> SomeException
SomeException (SessionDirDoesNotExistError -> SomeException)
-> SessionDirDoesNotExistError -> SomeException
forall a b. (a -> b) -> a -> b
$ FilePath -> SessionDirDoesNotExistError
ErrSessionDirDoesNotExist FilePath
sessionDir)
        (IO a -> IO a) -> (IO a -> IO a) -> IO a -> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SessionDirLockedError -> SomeException) -> IO a -> IO a
forall e1 e2 (m :: * -> *) a.
(Exception e1, Exception e2, MonadCatch m) =>
(e1 -> e2) -> m a -> m a
mapExceptionWithActionRegistry (\(LSMT.ErrSessionDirLocked FsErrorPath
_fsErrorPath) -> SessionDirLockedError -> SomeException
forall e. Exception e => e -> SomeException
SomeException (SessionDirLockedError -> SomeException)
-> SessionDirLockedError -> SomeException
forall a b. (a -> b) -> a -> b
$ FilePath -> SessionDirLockedError
ErrSessionDirLocked FilePath
sessionDir)
        (IO a -> IO a) -> (IO a -> IO a) -> IO a -> IO a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SessionDirCorruptedError -> SomeException) -> IO a -> IO a
forall e1 e2 (m :: * -> *) a.
(Exception e1, Exception e2, MonadCatch m) =>
(e1 -> e2) -> m a -> m a
mapExceptionWithActionRegistry (\(LSMT.ErrSessionDirCorrupted FsErrorPath
_fsErrorPath) -> SessionDirCorruptedError -> SomeException
forall e. Exception e => e -> SomeException
SomeException (SessionDirCorruptedError -> SomeException)
-> SessionDirCorruptedError -> SomeException
forall a b. (a -> b) -> a -> b
$ FilePath -> SessionDirCorruptedError
ErrSessionDirCorrupted FilePath
sessionDir)

{-------------------------------------------------------------------------------
   Table union
-------------------------------------------------------------------------------}

-- | A table union was constructed with two tables that are not compatible.
data TableUnionNotCompatibleError
    = ErrTableUnionHandleTypeMismatch
        -- | The index of the first table.
        !Int
        -- | The type of the filesystem handle of the first table.
        !TypeRep
        -- | The index of the second table.
        !Int
        -- | The type of the filesystem handle of the second table.
        !TypeRep
    | ErrTableUnionSessionMismatch
        -- | The index of the first table.
        !Int
        -- | The session directory of the first table.
        !FilePath
        -- | The index of the second table.
        !Int
        -- | The session directory of the second table.
        !FilePath
    deriving stock (Int -> TableUnionNotCompatibleError -> ShowS
[TableUnionNotCompatibleError] -> ShowS
TableUnionNotCompatibleError -> FilePath
(Int -> TableUnionNotCompatibleError -> ShowS)
-> (TableUnionNotCompatibleError -> FilePath)
-> ([TableUnionNotCompatibleError] -> ShowS)
-> Show TableUnionNotCompatibleError
forall a.
(Int -> a -> ShowS) -> (a -> FilePath) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> TableUnionNotCompatibleError -> ShowS
showsPrec :: Int -> TableUnionNotCompatibleError -> ShowS
$cshow :: TableUnionNotCompatibleError -> FilePath
show :: TableUnionNotCompatibleError -> FilePath
$cshowList :: [TableUnionNotCompatibleError] -> ShowS
showList :: [TableUnionNotCompatibleError] -> ShowS
Show, TableUnionNotCompatibleError
-> TableUnionNotCompatibleError -> Bool
(TableUnionNotCompatibleError
 -> TableUnionNotCompatibleError -> Bool)
-> (TableUnionNotCompatibleError
    -> TableUnionNotCompatibleError -> Bool)
-> Eq TableUnionNotCompatibleError
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: TableUnionNotCompatibleError
-> TableUnionNotCompatibleError -> Bool
== :: TableUnionNotCompatibleError
-> TableUnionNotCompatibleError -> Bool
$c/= :: TableUnionNotCompatibleError
-> TableUnionNotCompatibleError -> Bool
/= :: TableUnionNotCompatibleError
-> TableUnionNotCompatibleError -> Bool
Eq)
    deriving anyclass (Show TableUnionNotCompatibleError
Typeable TableUnionNotCompatibleError
(Typeable TableUnionNotCompatibleError,
 Show TableUnionNotCompatibleError) =>
(TableUnionNotCompatibleError -> SomeException)
-> (SomeException -> Maybe TableUnionNotCompatibleError)
-> (TableUnionNotCompatibleError -> FilePath)
-> Exception TableUnionNotCompatibleError
SomeException -> Maybe TableUnionNotCompatibleError
TableUnionNotCompatibleError -> FilePath
TableUnionNotCompatibleError -> SomeException
forall e.
(Typeable e, Show e) =>
(e -> SomeException)
-> (SomeException -> Maybe e) -> (e -> FilePath) -> Exception e
$ctoException :: TableUnionNotCompatibleError -> SomeException
toException :: TableUnionNotCompatibleError -> SomeException
$cfromException :: SomeException -> Maybe TableUnionNotCompatibleError
fromException :: SomeException -> Maybe TableUnionNotCompatibleError
$cdisplayException :: TableUnionNotCompatibleError -> FilePath
displayException :: TableUnionNotCompatibleError -> FilePath
Exception)

{- | Internal helper. Convert:

*   t'LSMT.TableUnionNotCompatibleError' to t'TableUnionNotCompatibleError';
-}
_convertTableUnionNotCompatibleError ::
    forall a.
    (Int -> FilePath) ->
    IO a ->
    IO a
_convertTableUnionNotCompatibleError :: forall a. (Int -> FilePath) -> IO a -> IO a
_convertTableUnionNotCompatibleError Int -> FilePath
sessionDirFor =
    (TableUnionNotCompatibleError -> TableUnionNotCompatibleError)
-> IO a -> IO a
forall e1 e2 (m :: * -> *) a.
(Exception e1, Exception e2, MonadCatch m) =>
(e1 -> e2) -> m a -> m a
mapExceptionWithActionRegistry ((TableUnionNotCompatibleError -> TableUnionNotCompatibleError)
 -> IO a -> IO a)
-> (TableUnionNotCompatibleError -> TableUnionNotCompatibleError)
-> IO a
-> IO a
forall a b. (a -> b) -> a -> b
$ \case
        LSMT.ErrTableUnionHandleTypeMismatch Int
i1 TypeRep
typeRep1 Int
i2 TypeRep
typeRep2 ->
            Int -> TypeRep -> Int -> TypeRep -> TableUnionNotCompatibleError
ErrTableUnionHandleTypeMismatch Int
i1 TypeRep
typeRep1 Int
i2 TypeRep
typeRep2
        LSMT.ErrTableUnionSessionMismatch Int
i1 FsErrorPath
_fsErrorPath1 Int
i2 FsErrorPath
_fsErrorPath2 ->
            Int -> FilePath -> Int -> FilePath -> TableUnionNotCompatibleError
ErrTableUnionSessionMismatch Int
i1 (Int -> FilePath
sessionDirFor Int
i1) Int
i2 (Int -> FilePath
sessionDirFor Int
i2)