module System.FS.BlockIO.Sim (
    fromHasFS
    -- * Initialisation helpers
  , simHasBlockIO
  , simHasBlockIO'
  , simErrorHasBlockIO
  , simErrorHasBlockIO'
  ) where

import           Control.Concurrent.Class.MonadMVar
import           Control.Concurrent.Class.MonadSTM.Strict
import           Control.Monad (void)
import           Control.Monad.Class.MonadThrow
import           Control.Monad.Primitive (PrimMonad)
import qualified Data.ByteString.Char8 as BS
import           System.FS.API as API
import qualified System.FS.API.Lazy as API
import qualified System.FS.API.Strict as API
import           System.FS.BlockIO.API (HasBlockIO (..), LockFileHandle (..),
                     LockMode (..))
import           System.FS.BlockIO.Serial
import           System.FS.CallStack (prettyCallStack)
import           System.FS.Sim.Error
import           System.FS.Sim.MockFS hiding (hClose, hOpen)
import           System.FS.Sim.STM

fromHasFS ::
     forall m. (MonadCatch m, MonadMVar m, PrimMonad m)
  => HasFS m HandleMock
  -> m (HasBlockIO m HandleMock)
fromHasFS :: forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m) =>
HasFS m HandleMock -> m (HasBlockIO m HandleMock)
fromHasFS HasFS m HandleMock
hfs =
    (Handle HandleMock -> Bool -> m ())
-> (Handle HandleMock
    -> FileOffset -> FileOffset -> Advice -> m ())
-> (Handle HandleMock -> FileOffset -> FileOffset -> m ())
-> (FsPath -> LockMode -> m (Maybe (LockFileHandle m)))
-> (Handle HandleMock -> m ())
-> (FsPath -> m ())
-> (FsPath -> FsPath -> m ())
-> HasFS m HandleMock
-> m (HasBlockIO m HandleMock)
forall (m :: * -> *) h.
(MonadThrow m, MonadMVar m, PrimMonad m, Eq h) =>
(Handle h -> Bool -> m ())
-> (Handle h -> FileOffset -> FileOffset -> Advice -> m ())
-> (Handle h -> FileOffset -> FileOffset -> m ())
-> (FsPath -> LockMode -> m (Maybe (LockFileHandle m)))
-> (Handle h -> m ())
-> (FsPath -> m ())
-> (FsPath -> FsPath -> m ())
-> HasFS m h
-> m (HasBlockIO m h)
serialHasBlockIO
      Handle HandleMock -> Bool -> m ()
forall {f :: * -> *} {p} {p}. Applicative f => p -> p -> f ()
hSetNoCache
      Handle HandleMock -> FileOffset -> FileOffset -> Advice -> m ()
forall {f :: * -> *} {p} {p} {p} {p}.
Applicative f =>
p -> p -> p -> p -> f ()
hAdvise
      Handle HandleMock -> FileOffset -> FileOffset -> m ()
forall {f :: * -> *} {p} {p} {p}.
Applicative f =>
p -> p -> p -> f ()
hAllocate
      (HasFS m HandleMock
-> FsPath -> LockMode -> m (Maybe (LockFileHandle m))
forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> FsPath -> LockMode -> m (Maybe (LockFileHandle m))
simTryLockFile HasFS m HandleMock
hfs)
      Handle HandleMock -> m ()
forall {f :: * -> *} {p}. Applicative f => p -> f ()
simHSynchronise
      FsPath -> m ()
forall {f :: * -> *} {p}. Applicative f => p -> f ()
simSynchroniseDirectory
      (HasFS m HandleMock -> FsPath -> FsPath -> m ()
forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> FsPath -> FsPath -> m ()
simCreateHardLink HasFS m HandleMock
hfs)
      HasFS m HandleMock
hfs
  where
    -- TODO: It should be possible for the implementations and simulation to
    -- throw an FsError when doing file I/O with misaligned byte arrays after
    -- hSetNoCache. Maybe they should? It might be nicest to move hSetNoCache
    -- into fs-api and fs-sim because we'd need access to the internals.
    hSetNoCache :: p -> p -> f ()
hSetNoCache p
_h p
_b = () -> f ()
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
    hAdvise :: p -> p -> p -> p -> f ()
hAdvise p
_ p
_ p
_ p
_ = () -> f ()
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
    hAllocate :: p -> p -> p -> f ()
hAllocate p
_ p
_ p
_ = () -> f ()
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

    -- Disk operations are durable by construction
    simHSynchronise :: p -> f ()
simHSynchronise p
_ = () -> f ()
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
    simSynchroniseDirectory :: p -> f ()
simSynchroniseDirectory p
_ = () -> f ()
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

-- | Lock files are reader\/writer locks.
--
-- We implement this using the content of the lock file. The content is a
-- counter, positive for readers and negaive (specifically -1) for writers.
-- There can be any number of readers, but only one writer. Writers can not
-- coexist with readers.
--
-- Warning: This implementation is not robust under concurrent use (because
-- operations on files are not atomic) but should be ok for casual use. A
-- proper implementation would need to be part of the underlying 'HasFS'
-- implementations.
--
-- Warning: regular file operations on the "locked" file, like 'hOpen' or
-- 'removeFile', will still work. 'simTryLockFile' only defines how multiple
-- lock acquisitions on the same file interact, not how lock acquisition
-- interacts with other file operations.
--
simTryLockFile ::
     forall m h. MonadThrow m
  => HasFS m h
  -> FsPath
  -> LockMode
  -> m (Maybe (LockFileHandle m))
simTryLockFile :: forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> FsPath -> LockMode -> m (Maybe (LockFileHandle m))
simTryLockFile HasFS m h
hfs FsPath
path LockMode
lockmode =
    HasFS m h
-> FsPath
-> OpenMode
-> (Handle h -> m (Maybe (LockFileHandle m)))
-> m (Maybe (LockFileHandle m))
forall (m :: * -> *) h a.
(HasCallStack, MonadThrow m) =>
HasFS m h -> FsPath -> OpenMode -> (Handle h -> m a) -> m a
API.withFile HasFS m h
hfs FsPath
path (AllowExisting -> OpenMode
ReadWriteMode AllowExisting
AllowExisting) ((Handle h -> m (Maybe (LockFileHandle m)))
 -> m (Maybe (LockFileHandle m)))
-> (Handle h -> m (Maybe (LockFileHandle m)))
-> m (Maybe (LockFileHandle m))
forall a b. (a -> b) -> a -> b
$ \Handle h
h -> do
      Int
n <- Handle h -> m Int
readCount Handle h
h
      case LockMode
lockmode of
        LockMode
SharedLock    | Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>= Int
0 -> do Handle h -> Int -> m ()
writeCount Handle h
h (Int
nInt -> Int -> Int
forall a. Num a => a -> a -> a
+Int
1)
                                     m (Maybe (LockFileHandle m))
mkLockFileHandle
        LockMode
ExclusiveLock | Int
n Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== Int
0 -> do Handle h -> Int -> m ()
writeCount Handle h
h (-Int
1)
                                     m (Maybe (LockFileHandle m))
mkLockFileHandle
        LockMode
_                      -> Maybe (LockFileHandle m) -> m (Maybe (LockFileHandle m))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (LockFileHandle m)
forall a. Maybe a
Nothing
  where
    mkLockFileHandle :: m (Maybe (LockFileHandle m))
mkLockFileHandle = do
      -- A lock file handle keeps open the file in read mode, such that a locked
      -- file contributes to the number of open file handles. The mock FS allows
      -- multiple readers and up to one writer to open the file concurrently.
      Handle h
h <- HasFS m h -> HasCallStack => FsPath -> OpenMode -> m (Handle h)
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => FsPath -> OpenMode -> m (Handle h)
API.hOpen HasFS m h
hfs FsPath
path OpenMode
ReadMode
      Maybe (LockFileHandle m) -> m (Maybe (LockFileHandle m))
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return (LockFileHandle m -> Maybe (LockFileHandle m)
forall a. a -> Maybe a
Just (LockFileHandle { hUnlock :: m ()
hUnlock = Handle h -> m ()
hUnlock Handle h
h }))

    hUnlock :: Handle h -> m ()
hUnlock Handle h
h0 =
      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
API.withFile HasFS m h
hfs FsPath
path (AllowExisting -> OpenMode
ReadWriteMode AllowExisting
AllowExisting) ((Handle h -> m ()) -> m ()) -> (Handle h -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \Handle h
h -> do
        Int
n <- Handle h -> m Int
readCount Handle h
h
        case LockMode
lockmode of
          LockMode
SharedLock    | Int
n Int -> Int -> Bool
forall a. Ord a => a -> a -> Bool
>  Int
0  -> Handle h -> Int -> m ()
writeCount Handle h
h (Int
nInt -> Int -> Int
forall a. Num a => a -> a -> a
-Int
1)
          LockMode
ExclusiveLock | Int
n Int -> Int -> Bool
forall a. Eq a => a -> a -> Bool
== -Int
1 -> Handle h -> Int -> m ()
writeCount Handle h
h Int
0
          LockMode
_                       -> FsError -> m ()
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO FsError
countCorrupt
        HasFS m h -> HasCallStack => Handle h -> m ()
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> m ()
hClose HasFS m h
hfs Handle h
h0

    readCount :: Handle h -> m Int
    readCount :: Handle h -> m Int
readCount Handle h
h = do
      ByteString
content <- ByteString -> ByteString
BS.toStrict (ByteString -> ByteString) -> m ByteString -> m ByteString
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HasFS m h -> Handle h -> AbsOffset -> m ByteString
forall (m :: * -> *) h.
Monad m =>
HasFS m h -> Handle h -> AbsOffset -> m ByteString
API.hGetAllAt HasFS m h
hfs Handle h
h AbsOffset
0
      case ReadS Int
forall a. Read a => ReadS a
reads (ByteString -> [Char]
BS.unpack ByteString
content) of
        [(Int, [Char])]
_ | ByteString -> Bool
BS.null ByteString
content -> Int -> m Int
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Int
0
        [(Int
n, [Char]
"")]           -> Int -> m Int
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Int
n
        [(Int, [Char])]
_                   -> FsError -> m Int
forall e a. Exception e => e -> m a
forall (m :: * -> *) e a. (MonadThrow m, Exception e) => e -> m a
throwIO FsError
countCorrupt

    writeCount :: Handle h -> Int -> m ()
    writeCount :: Handle h -> Int -> m ()
writeCount Handle h
h Int
n = do
      HasFS m h -> HasCallStack => Handle h -> SeekMode -> Int64 -> m ()
forall (m :: * -> *) h.
HasFS m h -> HasCallStack => Handle h -> SeekMode -> Int64 -> m ()
API.hSeek HasFS m h
hfs Handle h
h SeekMode
AbsoluteSeek Int64
0
      Word64
_ <- HasFS m h -> Handle h -> ByteString -> m Word64
forall (m :: * -> *) h.
(HasCallStack, Monad m) =>
HasFS m h -> Handle h -> ByteString -> m Word64
API.hPutAllStrict HasFS m h
hfs Handle h
h ([Char] -> ByteString
BS.pack (Int -> [Char]
forall a. Show a => a -> [Char]
show Int
n))
      () -> m ()
forall a. a -> m a
forall (m :: * -> *) a. Monad m => a -> m a
return ()

    countCorrupt :: FsError
countCorrupt =
      FsError {
        fsErrorType :: FsErrorType
fsErrorType   = FsErrorType
FsOther,
        fsErrorPath :: FsErrorPath
fsErrorPath   = FsPath -> FsErrorPath
fsToFsErrorPathUnmounted FsPath
path,
        fsErrorString :: [Char]
fsErrorString = [Char]
"lock file content corrupted",
        fsErrorNo :: Maybe Errno
fsErrorNo     = Maybe Errno
forall a. Maybe a
Nothing,
        fsErrorStack :: PrettyCallStack
fsErrorStack  = PrettyCallStack
HasCallStack => PrettyCallStack
prettyCallStack,
        fsLimitation :: Bool
fsLimitation  = Bool
False
      }

-- | @'simCreateHardLink' hfs source target@ creates a simulated hard link for
-- the @source@ path at the @target@ path.
--
-- The hard link is simulated by simply copying the source file to the target
-- path, which means that it should only be used to create hard links for files
-- that are not modified afterwards!
--
-- TODO: if we wanted to simulate proper hard links, we would have to bake the
-- feature into @fs-sim@.
simCreateHardLink :: MonadThrow m => HasFS m h -> FsPath -> FsPath -> m ()
simCreateHardLink :: forall (m :: * -> *) h.
MonadThrow m =>
HasFS m h -> FsPath -> FsPath -> m ()
simCreateHardLink HasFS m h
hfs FsPath
sourcePath FsPath
targetPath =
    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
API.withFile HasFS m h
hfs FsPath
sourcePath OpenMode
API.ReadMode ((Handle h -> m ()) -> m ()) -> (Handle h -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \Handle h
sourceHandle ->
    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
API.withFile HasFS m h
hfs FsPath
targetPath (AllowExisting -> OpenMode
API.WriteMode AllowExisting
API.MustBeNew) ((Handle h -> m ()) -> m ()) -> (Handle h -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \Handle h
targetHandle -> do
      -- This should /hopefully/ stream using lazy IO
      ByteString
bs <- HasFS m h -> Handle h -> m ByteString
forall (m :: * -> *) h.
Monad m =>
HasFS m h -> Handle h -> m ByteString
API.hGetAll HasFS m h
hfs Handle h
sourceHandle
      m Word64 -> m ()
forall (f :: * -> *) a. Functor f => f a -> f ()
void (m Word64 -> m ()) -> m Word64 -> m ()
forall a b. (a -> b) -> a -> b
$ HasFS m h -> Handle h -> ByteString -> m Word64
forall (m :: * -> *) h.
(HasCallStack, Monad m) =>
HasFS m h -> Handle h -> ByteString -> m Word64
API.hPutAll HasFS m h
hfs Handle h
targetHandle ByteString
bs

{-------------------------------------------------------------------------------
  Initialisation helpers
-------------------------------------------------------------------------------}

simHasBlockIO ::
     (MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m)
  => StrictTMVar m MockFS
  -> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simHasBlockIO :: forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m) =>
StrictTMVar m MockFS
-> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simHasBlockIO StrictTMVar m MockFS
var = do
    let hfs :: HasFS m HandleMock
hfs = StrictTMVar m MockFS -> HasFS m HandleMock
forall (m :: * -> *).
(MonadSTM m, MonadThrow m, PrimMonad m) =>
StrictTMVar m MockFS -> HasFS m HandleMock
simHasFS StrictTMVar m MockFS
var
    HasBlockIO m HandleMock
hbio <- HasFS m HandleMock -> m (HasBlockIO m HandleMock)
forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m) =>
HasFS m HandleMock -> m (HasBlockIO m HandleMock)
fromHasFS HasFS m HandleMock
hfs
    (HasFS m HandleMock, HasBlockIO m HandleMock)
-> m (HasFS m HandleMock, HasBlockIO m HandleMock)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (HasFS m HandleMock
hfs, HasBlockIO m HandleMock
hbio)

simHasBlockIO' ::
     (MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m)
  => MockFS
  -> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simHasBlockIO' :: forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m) =>
MockFS -> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simHasBlockIO' MockFS
mockFS = do
    HasFS m HandleMock
hfs <- MockFS -> m (HasFS m HandleMock)
forall (m :: * -> *).
(MonadSTM m, MonadThrow m, PrimMonad m) =>
MockFS -> m (HasFS m HandleMock)
simHasFS' MockFS
mockFS
    HasBlockIO m HandleMock
hbio <- HasFS m HandleMock -> m (HasBlockIO m HandleMock)
forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m) =>
HasFS m HandleMock -> m (HasBlockIO m HandleMock)
fromHasFS HasFS m HandleMock
hfs
    (HasFS m HandleMock, HasBlockIO m HandleMock)
-> m (HasFS m HandleMock, HasBlockIO m HandleMock)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (HasFS m HandleMock
hfs, HasBlockIO m HandleMock
hbio)

simErrorHasBlockIO ::
     forall m. (MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m)
  => StrictTMVar m MockFS
  -> StrictTVar m Errors
  -> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simErrorHasBlockIO :: forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m) =>
StrictTMVar m MockFS
-> StrictTVar m Errors
-> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simErrorHasBlockIO StrictTMVar m MockFS
fsVar StrictTVar m Errors
errorsVar = do
    let hfs :: HasFS m HandleMock
hfs = StrictTMVar m MockFS -> StrictTVar m Errors -> HasFS m HandleMock
forall (m :: * -> *).
(MonadSTM m, MonadThrow m, PrimMonad m) =>
StrictTMVar m MockFS -> StrictTVar m Errors -> HasFS m HandleMock
simErrorHasFS StrictTMVar m MockFS
fsVar StrictTVar m Errors
errorsVar
    HasBlockIO m HandleMock
hbio <- HasFS m HandleMock -> m (HasBlockIO m HandleMock)
forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m) =>
HasFS m HandleMock -> m (HasBlockIO m HandleMock)
fromHasFS HasFS m HandleMock
hfs
    (HasFS m HandleMock, HasBlockIO m HandleMock)
-> m (HasFS m HandleMock, HasBlockIO m HandleMock)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (HasFS m HandleMock
hfs, HasBlockIO m HandleMock
hbio)

simErrorHasBlockIO' ::
     (MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m)
  => MockFS
  -> Errors
  -> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simErrorHasBlockIO' :: forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m, MonadSTM m) =>
MockFS -> Errors -> m (HasFS m HandleMock, HasBlockIO m HandleMock)
simErrorHasBlockIO' MockFS
mockFS Errors
errs = do
    HasFS m HandleMock
hfs <- MockFS -> Errors -> m (HasFS m HandleMock)
forall (m :: * -> *).
(MonadSTM m, MonadThrow m, PrimMonad m) =>
MockFS -> Errors -> m (HasFS m HandleMock)
simErrorHasFS' MockFS
mockFS Errors
errs
    HasBlockIO m HandleMock
hbio <- HasFS m HandleMock -> m (HasBlockIO m HandleMock)
forall (m :: * -> *).
(MonadCatch m, MonadMVar m, PrimMonad m) =>
HasFS m HandleMock -> m (HasBlockIO m HandleMock)
fromHasFS HasFS m HandleMock
hfs
    (HasFS m HandleMock, HasBlockIO m HandleMock)
-> m (HasFS m HandleMock, HasBlockIO m HandleMock)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (HasFS m HandleMock
hfs, HasBlockIO m HandleMock
hbio)