{-# LANGUAGE TypeFamilies  #-}
{-# LANGUAGE UnboxedTuples #-}

-- | Abstract interface, types, and utilities.
module System.FS.BlockIO.API (
    -- * HasBlockIO
    HasBlockIO (..)
  , IOOp (..)
  , ioopHandle
  , ioopFileOffset
  , ioopBuffer
  , ioopBufferOffset
  , ioopByteCount
  , IOResult (..)
    -- ** Advice
  , Advice (..)
  , hAdviseAll
  , hDropCacheAll
    -- ** File locks
  , GHC.LockMode (..)
  , GHC.FileLockingNotSupported (..)
  , LockFileHandle (..)
    -- ** Storage synchronisation
  , synchroniseFile
  , synchroniseDirectoryRecursive
    -- * Re-exports
  , ByteCount
  , FileOffset
  ) where

import           Control.DeepSeq
import           Control.Monad (forM_)
import           Control.Monad.Class.MonadThrow (MonadThrow (..))
import           Control.Monad.Primitive (PrimMonad (PrimState))
import           Data.Primitive.ByteArray (MutableByteArray)
import qualified Data.Vector as V
import qualified Data.Vector.Generic as VG
import qualified Data.Vector.Generic.Mutable as VGM
import qualified Data.Vector.Primitive as VP
import qualified Data.Vector.Unboxed as VU
import qualified Data.Vector.Unboxed.Mutable as VUM
import qualified GHC.IO.Handle.Lock as GHC
import           GHC.Stack (HasCallStack)
import qualified System.FS.API as FS
import           System.FS.API (BufferOffset, FsPath, Handle (..), HasFS)
import           System.Posix.Types (ByteCount, FileOffset)
import           Text.Printf

-- | Abstract interface for submitting large batches of I\/O operations. This
-- interface is an extension of the 'HasFS' interface that is provided by the
-- @fs-api@ package.
--
-- The interface tries to specify uniform behaviour, but each implementation can
-- have subtly different effects for a variety of reasons. However, for the most
-- part the underlying implementation of an instance of the interface should not
-- affect the correctness of programs that use the interface.
--
-- For uniform behaviour across implementations, functions that create a new
-- instance of the interface should initialise an IO context. This IO context
-- may be of any shape, as long as the context has two modes: open and closed.
-- This context is only important for the 'close' and 'submitIO' functions. As
-- long as the IO context is open, 'submitIO' should perform batches of I\/O
-- operations as expected, but 'submitIO' should throw an error as soon as the
-- IO context is closed. Once the IO context is closed, it can not be re-opened
-- again. Instead, the user should create a new instance of the interface.
--
-- Note: there are a bunch of functions in the interface that have nothing to do
-- with submitting large batches of I\/O operations. In fact, only 'close' and
-- 'submitIO' are related to that. All other functions were put in this record
-- for simplicity because the authors of the library needed them and it was more
-- straightforward to add them here then to add them to @fs-api@. Still these
-- unrelated functions could and should all be moved into @fs-api@ at some point
-- in the future.
--
data HasBlockIO m h = HasBlockIO {
    -- | (Idempotent) close the IO context that is required for running
    -- 'submitIO'.
    --
    -- Using 'submitIO' after 'close' throws an 'FsError' exception.
    --
    forall (m :: * -> *) h. HasBlockIO m h -> HasCallStack => m ()
close    :: HasCallStack => m ()

    -- | Submit a batch of I\/O operations and wait for the result.
    --
    -- Results correspond to input 'IOOp's in a pair-wise manner, i.e., one can
    -- match 'IOOp's with 'IOResult's by indexing into both vectors at the same
    -- position.
    --
    -- If any of the I\/O operations fails, an 'FsError' exception will be thrown.
    --
    -- The buffers in the 'IOOp's should be pinned memory. If any buffer is
    -- unpinned memory, an 'FsError' exception will be thrown.
    --
  , forall (m :: * -> *) h.
HasBlockIO m h
-> HasCallStack =>
   Vector (IOOp (PrimState m) h) -> m (Vector IOResult)
submitIO :: HasCallStack => V.Vector (IOOp (PrimState m) h) -> m (VU.Vector IOResult)

    -- TODO: once file caching is disabled, subsequent reads/writes with
    -- misaligned byte arrays should throw an error. Preferably, this should
    -- happen in both the simulation and real implementation, even if the real
    -- implementation does not support setting the file caching mode. This would
    -- make the behaviour of the file caching mode more uniform across
    -- implementations and platforms.

    -- | Set the file data caching mode for a file handle.
    --
  , forall (m :: * -> *) h. HasBlockIO m h -> Handle h -> Bool -> m ()
hSetNoCache :: Handle h -> Bool -> m ()

    -- | Predeclare an access pattern for file data.
    --
  , forall (m :: * -> *) h.
HasBlockIO m h
-> Handle h -> FileOffset -> FileOffset -> Advice -> m ()
hAdvise :: Handle h -> FileOffset -> FileOffset -> Advice -> m ()

    -- | Allocate file space.
    --
  , forall (m :: * -> *) h.
HasBlockIO m h -> Handle h -> FileOffset -> FileOffset -> m ()
hAllocate :: Handle h -> FileOffset -> FileOffset -> m ()

    -- NOTE: though it would have been nicer to allow locking /file handles/
    -- instead of /file paths/, it would make the implementation of this
    -- function in 'IO' much more complex. In particular, if we want to reuse
    -- "GHC.IO.Handle.Lock" functionality, then we have to either ...
    --
    -- 1. Convert there and back between OS-specific file descriptors and
    --   'GHC.Handle's, which is not possible on Windows without creating new
    --   file descriptors, or ...
    -- 2. Vendor all of the "GHC.IO.Handle.Lock" code and its dependencies
    --    (i.e., modules), which is a prohibitively large body of code for GHC
    --    versions before @9.0@.
    --
    -- The current interface is therefore limited, but should be sufficient for
    -- use cases where a lock file is used to guard against concurrent access by
    -- different processes. e.g., a database lock file.
    --
    -- TODO: it is /probably/ possible to provide a 'onLockFileHandle' function
    -- that allows you to use 'LockFileHandle' as a 'Handle', but only within a
    -- limited scope. That is, it has to fit the style of @withHandleToHANDLE ::
    -- Handle -> (HANDLE -> IO a) -> IO a@ from the @Win32@ package.

    -- | Try to acquire a file lock without blocking.
    --
    -- This function throws 'GHC.FileLockingNotSupported' when file locking is
    -- not supported.
    --
  , forall (m :: * -> *) h.
HasBlockIO m h
-> FsPath -> LockMode -> m (Maybe (LockFileHandle m))
tryLockFile :: FsPath -> GHC.LockMode -> m (Maybe (LockFileHandle m))

    -- | Synchronise file contents with the storage device.
    --
    -- This ensures that all changes to the file handle's contents, which might
    -- exist only in memory as buffered system cache pages, are
    -- transferred/flushed to disk. This will also update the file handle's
    -- associated metadata.
    --
  , forall (m :: * -> *) h. HasBlockIO m h -> Handle h -> m ()
hSynchronise :: Handle h -> m ()

    -- | Synchronise a directory with the storage device.
    --
    -- This ensures that all changes to the directory, which might exist only in
    -- memory as buffered changes, are transferred/flushed to disk. This will
    -- also update the directory's associated metadata.
    --
  , forall (m :: * -> *) h. HasBlockIO m h -> FsPath -> m ()
synchroniseDirectory :: FsPath -> m ()

    -- | Create a hard link for an existing file at the source path and a new
    -- file at the target path.
    --
  , forall (m :: * -> *) h. HasBlockIO m h -> FsPath -> FsPath -> m ()
createHardLink :: FsPath -> FsPath -> m ()
  }

instance NFData (HasBlockIO m h) where
  rnf :: HasBlockIO m h -> ()
rnf (HasBlockIO HasCallStack => m ()
a HasCallStack =>
Vector (IOOp (PrimState m) h) -> m (Vector IOResult)
b Handle h -> Bool -> m ()
c Handle h -> FileOffset -> FileOffset -> Advice -> m ()
d Handle h -> FileOffset -> FileOffset -> m ()
e FsPath -> LockMode -> m (Maybe (LockFileHandle m))
f Handle h -> m ()
g FsPath -> m ()
h FsPath -> FsPath -> m ()
i) =
      m () -> ()
forall a. a -> ()
rwhnf m ()
HasCallStack => m ()
a () -> () -> ()
forall a b. a -> b -> b
`seq` (Vector (IOOp (PrimState m) h) -> m (Vector IOResult)) -> ()
forall a. a -> ()
rwhnf HasCallStack =>
Vector (IOOp (PrimState m) h) -> m (Vector IOResult)
Vector (IOOp (PrimState m) h) -> m (Vector IOResult)
b () -> () -> ()
forall a b. a -> b -> b
`seq` (Handle h -> Bool -> m ()) -> ()
forall a. NFData a => a -> ()
rnf Handle h -> Bool -> m ()
c () -> () -> ()
forall a b. a -> b -> b
`seq`
      (Handle h -> FileOffset -> FileOffset -> Advice -> m ()) -> ()
forall a. a -> ()
rwhnf Handle h -> FileOffset -> FileOffset -> Advice -> m ()
d () -> () -> ()
forall a b. a -> b -> b
`seq` (Handle h -> FileOffset -> FileOffset -> m ()) -> ()
forall a. a -> ()
rwhnf Handle h -> FileOffset -> FileOffset -> m ()
e () -> () -> ()
forall a b. a -> b -> b
`seq` (FsPath -> LockMode -> m (Maybe (LockFileHandle m))) -> ()
forall a. a -> ()
rwhnf FsPath -> LockMode -> m (Maybe (LockFileHandle m))
f () -> () -> ()
forall a b. a -> b -> b
`seq`
      (Handle h -> m ()) -> ()
forall a. a -> ()
rwhnf Handle h -> m ()
g () -> () -> ()
forall a b. a -> b -> b
`seq` (FsPath -> m ()) -> ()
forall a. a -> ()
rwhnf FsPath -> m ()
h () -> () -> ()
forall a b. a -> b -> b
`seq` (FsPath -> FsPath -> m ()) -> ()
forall a. a -> ()
rwhnf FsPath -> FsPath -> m ()
i

data IOOp s h =
    IOOpRead  !(Handle h) !FileOffset !(MutableByteArray s) !BufferOffset !ByteCount
  | IOOpWrite !(Handle h) !FileOffset !(MutableByteArray s) !BufferOffset !ByteCount

instance NFData (IOOp s h) where
  rnf :: IOOp s h -> ()
rnf = IOOp s h -> ()
forall a. a -> ()
rwhnf

ioopHandle :: IOOp s h -> Handle h
ioopHandle :: forall s h. IOOp s h -> Handle h
ioopHandle (IOOpRead Handle h
h FileOffset
_ MutableByteArray s
_ BufferOffset
_ ByteCount
_)  = Handle h
h
ioopHandle (IOOpWrite Handle h
h FileOffset
_ MutableByteArray s
_ BufferOffset
_ ByteCount
_) = Handle h
h

ioopFileOffset :: IOOp s h -> FileOffset
ioopFileOffset :: forall s h. IOOp s h -> FileOffset
ioopFileOffset (IOOpRead Handle h
_ FileOffset
off MutableByteArray s
_ BufferOffset
_ ByteCount
_)  = FileOffset
off
ioopFileOffset (IOOpWrite Handle h
_ FileOffset
off MutableByteArray s
_ BufferOffset
_ ByteCount
_) = FileOffset
off

ioopBuffer :: IOOp s h -> MutableByteArray s
ioopBuffer :: forall s h. IOOp s h -> MutableByteArray s
ioopBuffer (IOOpRead Handle h
_ FileOffset
_ MutableByteArray s
buf BufferOffset
_ ByteCount
_)  = MutableByteArray s
buf
ioopBuffer (IOOpWrite Handle h
_ FileOffset
_ MutableByteArray s
buf BufferOffset
_ ByteCount
_) = MutableByteArray s
buf

ioopBufferOffset :: IOOp s h -> BufferOffset
ioopBufferOffset :: forall s h. IOOp s h -> BufferOffset
ioopBufferOffset (IOOpRead Handle h
_ FileOffset
_ MutableByteArray s
_ BufferOffset
bufOff ByteCount
_)  = BufferOffset
bufOff
ioopBufferOffset (IOOpWrite Handle h
_ FileOffset
_ MutableByteArray s
_ BufferOffset
bufOff ByteCount
_) = BufferOffset
bufOff

ioopByteCount :: IOOp s h -> ByteCount
ioopByteCount :: forall s h. IOOp s h -> ByteCount
ioopByteCount (IOOpRead Handle h
_ FileOffset
_ MutableByteArray s
_ BufferOffset
_ ByteCount
c)  = ByteCount
c
ioopByteCount (IOOpWrite Handle h
_ FileOffset
_ MutableByteArray s
_ BufferOffset
_ ByteCount
c) = ByteCount
c

-- | Number of read/written bytes.
newtype IOResult = IOResult ByteCount
  deriving stock (Int -> IOResult -> ShowS
[IOResult] -> ShowS
IOResult -> String
(Int -> IOResult -> ShowS)
-> (IOResult -> String) -> ([IOResult] -> ShowS) -> Show IOResult
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> IOResult -> ShowS
showsPrec :: Int -> IOResult -> ShowS
$cshow :: IOResult -> String
show :: IOResult -> String
$cshowList :: [IOResult] -> ShowS
showList :: [IOResult] -> ShowS
Show, IOResult -> IOResult -> Bool
(IOResult -> IOResult -> Bool)
-> (IOResult -> IOResult -> Bool) -> Eq IOResult
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: IOResult -> IOResult -> Bool
== :: IOResult -> IOResult -> Bool
$c/= :: IOResult -> IOResult -> Bool
/= :: IOResult -> IOResult -> Bool
Eq)
  deriving newtype Addr# -> Int# -> IOResult
ByteArray# -> Int# -> IOResult
Proxy IOResult -> Int#
IOResult -> Int#
(Proxy IOResult -> Int#)
-> (IOResult -> Int#)
-> (Proxy IOResult -> Int#)
-> (IOResult -> Int#)
-> (ByteArray# -> Int# -> IOResult)
-> (forall s.
    MutableByteArray# s
    -> Int# -> State# s -> (# State# s, IOResult #))
-> (forall s.
    MutableByteArray# s -> Int# -> IOResult -> State# s -> State# s)
-> (forall s.
    MutableByteArray# s
    -> Int# -> Int# -> IOResult -> State# s -> State# s)
-> (Addr# -> Int# -> IOResult)
-> (forall s.
    Addr# -> Int# -> State# s -> (# State# s, IOResult #))
-> (forall s. Addr# -> Int# -> IOResult -> State# s -> State# s)
-> (forall s.
    Addr# -> Int# -> Int# -> IOResult -> State# s -> State# s)
-> Prim IOResult
forall s. Addr# -> Int# -> Int# -> IOResult -> State# s -> State# s
forall s. Addr# -> Int# -> State# s -> (# State# s, IOResult #)
forall s. Addr# -> Int# -> IOResult -> State# s -> State# s
forall s.
MutableByteArray# s
-> Int# -> Int# -> IOResult -> State# s -> State# s
forall s.
MutableByteArray# s -> Int# -> State# s -> (# State# s, IOResult #)
forall s.
MutableByteArray# s -> Int# -> IOResult -> State# s -> State# s
forall a.
(Proxy a -> Int#)
-> (a -> Int#)
-> (Proxy a -> Int#)
-> (a -> Int#)
-> (ByteArray# -> Int# -> a)
-> (forall s.
    MutableByteArray# s -> Int# -> State# s -> (# State# s, a #))
-> (forall s.
    MutableByteArray# s -> Int# -> a -> State# s -> State# s)
-> (forall s.
    MutableByteArray# s -> Int# -> Int# -> a -> State# s -> State# s)
-> (Addr# -> Int# -> a)
-> (forall s. Addr# -> Int# -> State# s -> (# State# s, a #))
-> (forall s. Addr# -> Int# -> a -> State# s -> State# s)
-> (forall s. Addr# -> Int# -> Int# -> a -> State# s -> State# s)
-> Prim a
$csizeOfType# :: Proxy IOResult -> Int#
sizeOfType# :: Proxy IOResult -> Int#
$csizeOf# :: IOResult -> Int#
sizeOf# :: IOResult -> Int#
$calignmentOfType# :: Proxy IOResult -> Int#
alignmentOfType# :: Proxy IOResult -> Int#
$calignment# :: IOResult -> Int#
alignment# :: IOResult -> Int#
$cindexByteArray# :: ByteArray# -> Int# -> IOResult
indexByteArray# :: ByteArray# -> Int# -> IOResult
$creadByteArray# :: forall s.
MutableByteArray# s -> Int# -> State# s -> (# State# s, IOResult #)
readByteArray# :: forall s.
MutableByteArray# s -> Int# -> State# s -> (# State# s, IOResult #)
$cwriteByteArray# :: forall s.
MutableByteArray# s -> Int# -> IOResult -> State# s -> State# s
writeByteArray# :: forall s.
MutableByteArray# s -> Int# -> IOResult -> State# s -> State# s
$csetByteArray# :: forall s.
MutableByteArray# s
-> Int# -> Int# -> IOResult -> State# s -> State# s
setByteArray# :: forall s.
MutableByteArray# s
-> Int# -> Int# -> IOResult -> State# s -> State# s
$cindexOffAddr# :: Addr# -> Int# -> IOResult
indexOffAddr# :: Addr# -> Int# -> IOResult
$creadOffAddr# :: forall s. Addr# -> Int# -> State# s -> (# State# s, IOResult #)
readOffAddr# :: forall s. Addr# -> Int# -> State# s -> (# State# s, IOResult #)
$cwriteOffAddr# :: forall s. Addr# -> Int# -> IOResult -> State# s -> State# s
writeOffAddr# :: forall s. Addr# -> Int# -> IOResult -> State# s -> State# s
$csetOffAddr# :: forall s. Addr# -> Int# -> Int# -> IOResult -> State# s -> State# s
setOffAddr# :: forall s. Addr# -> Int# -> Int# -> IOResult -> State# s -> State# s
VP.Prim

newtype instance VUM.MVector s IOResult = MV_IOResult (VP.MVector s IOResult)
newtype instance VU.Vector     IOResult = V_IOResult  (VP.Vector    IOResult)

deriving via (VU.UnboxViaPrim IOResult) instance VGM.MVector VU.MVector IOResult
deriving via (VU.UnboxViaPrim IOResult) instance VG.Vector   VU.Vector  IOResult

instance VUM.Unbox IOResult

{-------------------------------------------------------------------------------
  Advice
-------------------------------------------------------------------------------}

-- | Copy of "System.Posix.Fcntl.Advice" from the @unix@ package
data Advice =
    AdviceNormal
  | AdviceRandom
  | AdviceSequential
  | AdviceWillNeed
  | AdviceDontNeed
  | AdviceNoReuse
  deriving stock (Int -> Advice -> ShowS
[Advice] -> ShowS
Advice -> String
(Int -> Advice -> ShowS)
-> (Advice -> String) -> ([Advice] -> ShowS) -> Show Advice
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> Advice -> ShowS
showsPrec :: Int -> Advice -> ShowS
$cshow :: Advice -> String
show :: Advice -> String
$cshowList :: [Advice] -> ShowS
showList :: [Advice] -> ShowS
Show, Advice -> Advice -> Bool
(Advice -> Advice -> Bool)
-> (Advice -> Advice -> Bool) -> Eq Advice
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: Advice -> Advice -> Bool
== :: Advice -> Advice -> Bool
$c/= :: Advice -> Advice -> Bool
/= :: Advice -> Advice -> Bool
Eq)

-- | Apply 'Advice' to all bytes of a file, which is referenced by a 'Handle'.
hAdviseAll :: HasBlockIO m h -> Handle h -> Advice -> m ()
hAdviseAll :: forall (m :: * -> *) h.
HasBlockIO m h -> Handle h -> Advice -> m ()
hAdviseAll HasBlockIO m h
hbio Handle h
h Advice
advice = HasBlockIO m h
-> Handle h -> FileOffset -> FileOffset -> Advice -> m ()
forall (m :: * -> *) h.
HasBlockIO m h
-> Handle h -> FileOffset -> FileOffset -> Advice -> m ()
hAdvise HasBlockIO m h
hbio Handle h
h FileOffset
0 FileOffset
0 Advice
advice -- len=0 implies until the end of file

-- | Drop the full file referenced by a 'Handle' from the OS page cache, if
-- present.
hDropCacheAll :: HasBlockIO m h -> Handle h -> m ()
hDropCacheAll :: forall (m :: * -> *) h. HasBlockIO m h -> Handle h -> m ()
hDropCacheAll HasBlockIO m h
hbio Handle h
h = HasBlockIO m h -> Handle h -> Advice -> m ()
forall (m :: * -> *) h.
HasBlockIO m h -> Handle h -> Advice -> m ()
hAdviseAll HasBlockIO m h
hbio Handle h
h Advice
AdviceDontNeed

{-------------------------------------------------------------------------------
  Storage synchronisation
-------------------------------------------------------------------------------}

{-# SPECIALISE synchroniseFile :: HasFS IO h -> HasBlockIO IO h -> FsPath -> IO () #-}
-- | Synchronise a file's contents and metadata with the storage device.
synchroniseFile :: MonadThrow m => HasFS m h -> HasBlockIO m h -> FsPath -> m ()
synchroniseFile :: forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> HasBlockIO m h -> FsPath -> m ()
synchroniseFile HasFS m h
hfs HasBlockIO m h
hbio FsPath
path =
    HasFS m h -> FsPath -> OpenMode -> (Handle h -> m ()) -> m ()
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
FS.withFile HasFS m h
hfs FsPath
path (AllowExisting -> OpenMode
FS.ReadWriteMode AllowExisting
FS.MustExist) ((Handle h -> m ()) -> m ()) -> (Handle h -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ HasBlockIO m h -> Handle h -> m ()
forall (m :: * -> *) h. HasBlockIO m h -> Handle h -> m ()
hSynchronise HasBlockIO m h
hbio

{-# SPECIALISE synchroniseDirectoryRecursive ::
     HasFS IO h
  -> HasBlockIO IO h
  -> FsPath
  -> IO ()
  #-}
-- | Synchronise a directory's contents and metadata with the storage device,
-- and recursively for all entries in the directory.
synchroniseDirectoryRecursive ::
     MonadThrow m
  => HasFS m h
  -> HasBlockIO m h
  -> FsPath
  -> m ()
synchroniseDirectoryRecursive :: forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> HasBlockIO m h -> FsPath -> m ()
synchroniseDirectoryRecursive HasFS m h
hfs HasBlockIO m h
hbio FsPath
path = do
    Set String
entries <- HasFS m h -> HasCallStack => FsPath -> m (Set String)
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> m (Set String)
FS.listDirectory HasFS m h
hfs FsPath
path
    Set String -> (String -> m ()) -> m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
t a -> (a -> m b) -> m ()
forM_ Set String
entries ((String -> m ()) -> m ()) -> (String -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \String
entry -> do
      let path' :: FsPath
path' = FsPath
path FsPath -> FsPath -> FsPath
FS.</> [String] -> FsPath
FS.mkFsPath [String
entry]
      Bool
isFile <- HasFS m h -> HasCallStack => FsPath -> m Bool
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> m Bool
FS.doesFileExist HasFS m h
hfs FsPath
path'
      if Bool
isFile then
        HasFS m h -> HasBlockIO m h -> FsPath -> m ()
forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> HasBlockIO m h -> FsPath -> m ()
synchroniseFile HasFS m h
hfs HasBlockIO m h
hbio FsPath
path'
      else do
        Bool
isDirectory <- HasFS m h -> HasCallStack => FsPath -> m Bool
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> m Bool
FS.doesDirectoryExist HasFS m h
hfs FsPath
path'
        if Bool
isDirectory then do
          HasFS m h -> HasBlockIO m h -> FsPath -> m ()
forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> HasBlockIO m h -> FsPath -> m ()
synchroniseDirectoryRecursive HasFS m h
hfs HasBlockIO m h
hbio FsPath
path'
          HasBlockIO m h -> FsPath -> m ()
forall (m :: * -> *) h. HasBlockIO m h -> FsPath -> m ()
synchroniseDirectory HasBlockIO m h
hbio FsPath
path'
        else
          String -> m ()
forall a. HasCallStack => String -> a
error (String -> m ()) -> String -> m ()
forall a b. (a -> b) -> a -> b
$ String -> ShowS
forall r. PrintfType r => String -> r
printf
            String
"listDirectoryRecursive: %s is not a file or directory"
            (FsPath -> String
forall a. Show a => a -> String
show FsPath
path')

{-------------------------------------------------------------------------------
  File locks
-------------------------------------------------------------------------------}

-- | A handle to a file locked using 'tryLockFile'.
newtype LockFileHandle m = LockFileHandle {
    -- | Release a file lock acquired using 'tryLockFile'.
    forall (m :: * -> *). LockFileHandle m -> m ()
hUnlock :: m ()
  }