-- Copyright (c) 2014-present, Facebook, Inc.
-- All rights reserved.
--
-- This source code is distributed under the terms of a BSD license,
-- found in the LICENSE file.

{- TODO

- do EVENTLOG stuff, track the data fetch numbers for performFetch

- timing: we should be using clock_gettime(CLOCK_MONOTONIC) instead of
  getCurrentTime, which will be affected by NTP and leap seconds.

- write different scheduling policies

-}

{-# LANGUAGE BangPatterns #-}
{-# LANGUAGE CPP #-}
{-# LANGUAGE ConstraintKinds #-}
{-# LANGUAGE ExistentialQuantification #-}
{-# LANGUAGE FlexibleInstances #-}
{-# LANGUAGE GADTs #-}
{-# LANGUAGE MultiWayIf #-}
{-# LANGUAGE OverloadedStrings #-}
{-# LANGUAGE PatternGuards #-}
{-# LANGUAGE RankNTypes #-}
{-# LANGUAGE RecordWildCards #-}
{-# LANGUAGE ScopedTypeVariables #-}
{-# LANGUAGE TypeFamilies #-}

-- |
-- The implementation of the 'Haxl' monad.  Most users should
-- import "Haxl.Core" instead of importing this module directly.
--
module Haxl.Core.Monad
  (
    -- * The monad
    GenHaxl(..)
  , Result(..)

    -- * Writes (for debugging only)
  , WriteTree(..)
  , tellWrite
  , tellWriteNoMemo
  , write
  , writeNoMemo
  , flattenWT
  , appendWTs
  , mbModifyWLRef
  , mapWrites

    -- * Cont
  , Cont(..)
  , toHaxl

    -- * IVar
  , IVar(..)
  , IVarContents(..)
  , newIVar
  , newFullIVar
  , withCurrentCCS
  , getIVar
  , getIVarWithWrites
  , putIVar

    -- * ResultVal
  , ResultVal(..)
  , done
  , eitherToResult
  , eitherToResultThrowIO

    -- * CompleteReq
  , CompleteReq(..)

    -- * Env
  , Env(..)
  , DataCacheItem(..)
  , DataCacheLookup(..)
  , HaxlDataCache
  , Caches
  , caches
  , initEnvWithData
  , initEnv
  , emptyEnv
  , env, withEnv
  , nextCallId
  , sanitizeEnv

    -- * Profiling
  , ProfileCurrent(..)

    -- * JobList
  , JobList(..)
  , appendJobList
  , lengthJobList
  , addJob

    -- * Exceptions
  , throw
  , raise
  , catch
  , catchIf
  , try
  , tryToHaxlException

    -- * Dumping the cache
  , dumpCacheAsHaskell
  , dumpCacheAsHaskellFn

    -- * CallGraph
#ifdef PROFILING
  , withCallGraph
#endif

    -- * Unsafe operations
  ,  unsafeLiftIO, unsafeToHaxlException
  ) where

import Haxl.Core.Flags
import Haxl.Core.Stats
import Haxl.Core.StateStore
import Haxl.Core.Exception
import Haxl.Core.RequestStore as RequestStore
import Haxl.Core.DataCache as DataCache
import Haxl.Core.Util (trace_)

import Control.Applicative (liftA2)
import Control.Arrow (left)
import Control.Concurrent.STM
import qualified Control.Monad.Catch as Catch
import Control.Exception (Exception(..), SomeException, throwIO)
#if __GLASGOW_HASKELL__ >= 808
import Control.Monad hiding (MonadFail)
import qualified Control.Monad as CTL
#else
import Control.Monad
#endif
import qualified Control.Exception as Exception
import Data.Either (rights)
import Data.IORef
import Data.Int
import Data.List.NonEmpty (NonEmpty(..))
import qualified Data.List.NonEmpty as NonEmpty
#if __GLASGOW_HASKELL__ < 804
import Data.Semigroup
#endif
import qualified Data.Text as Text
import Data.Typeable
import GHC.Exts (IsString(..))
import Text.PrettyPrint hiding ((<>))
import Text.Printf
#ifdef EVENTLOG
import Control.Exception (bracket_)
import Debug.Trace (traceEventIO)
#endif

#ifdef PROFILING
import qualified Data.Map as Map
import Data.Text (Text)
import Foreign.Ptr (Ptr)
import GHC.Stack
import Haxl.Core.CallGraph
#endif

-- -----------------------------------------------------------------------------
-- The environment

-- | The data we carry around in the Haxl monad.

data DataCacheItem u w a = DataCacheItem (IVar u w a) {-# UNPACK #-} !CallId
type HaxlDataCache u w = DataCache (DataCacheItem u w)
newtype DataCacheLookup w =
  DataCacheLookup
    (forall req a . Typeable (req a)
    => req a
    -> IO (Maybe (ResultVal a w)))

data Env u w = Env
  { Env u w -> HaxlDataCache u w
dataCache     :: {-# UNPACK #-} !(HaxlDataCache u w)
      -- ^ cached data fetches

  , Env u w -> HaxlDataCache u w
memoCache      :: {-# UNPACK #-} !(HaxlDataCache u w)
      -- ^ memoized computations

  , Env u w -> CallId
memoKey    :: {-# UNPACK #-} !CallId
      -- ^ current running memo key

  , Env u w -> Flags
flags        :: !Flags
      -- conservatively not unpacking, because this is passed
      -- to 'fetch' and would need to be rebuilt.

  , Env u w -> u
userEnv      :: u
      -- ^ user-supplied data, retrievable with 'env'

  , Env u w -> IORef Stats
statsRef     :: {-# UNPACK #-} !(IORef Stats)
      -- ^ statistics, collected according to the 'report' level in 'flags'.

  , Env u w -> IORef CallId
statsBatchIdRef :: {-# UNPACK #-} !(IORef Int)
     -- ^ keeps track of a Unique ID for each batch dispatched with stats
     -- enabled, for aggregating after.

  , Env u w -> IORef CallId
callIdRef :: {-# UNPACK #-} !(IORef CallId)
     -- ^ keeps track of a Unique ID for each fetch/memo.

  , Env u w -> ProfileCurrent
profCurrent    :: ProfileCurrent
     -- ^ current profiling label, see 'withLabel'

  , Env u w -> IORef Profile
profRef      :: {-# UNPACK #-} !(IORef Profile)
      -- ^ profiling data, collected according to the 'report' level in 'flags'.

  , Env u w -> StateStore
states       :: StateStore
      -- ^ Data sources and other components can store their state in
      -- here. Items in this store must be instances of 'StateKey'.

  , Env u w -> IORef (RequestStore u)
reqStoreRef :: {-# UNPACK #-} !(IORef (RequestStore u))
       -- ^ The set of requests that we have not submitted to data sources yet.
       -- Owned by the scheduler.

  , Env u w -> IORef (JobList u w)
runQueueRef :: {-# UNPACK #-} !(IORef (JobList u w))
       -- ^ runnable computations. Things get added to here when we wake up
       -- a computation that was waiting for something.  When the list is
       -- empty, either we're finished, or we're waiting for some data fetch
       -- to return.

  , Env u w -> IORef ReqCountMap
submittedReqsRef :: {-# UNPACK #-} !(IORef ReqCountMap)
       -- ^ all outgone fetches which haven't yet returned. Entries are
       -- removed from this map as the fetches finish. This field is
       -- useful for tracking outgone fetches to detect downstream
       -- failures.

  , Env u w -> TVar [CompleteReq u w]
completions :: {-# UNPACK #-} !(TVar [CompleteReq u w])
       -- ^ Requests that have completed.  Modified by data sources
       -- (via putResult) and the scheduler.  Waiting for this list to
       -- become non-empty is how the scheduler blocks waiting for
       -- data fetches to return.

  , Env u w -> IORef (WriteTree w)
writeLogsRef :: {-# UNPACK #-} !(IORef (WriteTree w))
       -- ^ A log of all writes done as part of this haxl computation. Any
       -- haxl computation that needs to be memoized runs in its own
       -- environment so that we can get a hold of those writes and put them
       -- in the IVar associated with the compuatation.
  , Env u w -> IORef (WriteTree w)
writeLogsRefNoMemo :: {-# UNPACK #-} !(IORef (WriteTree w))
       -- ^ This is just a specialized version of @writeLogsRef@, where we put
       -- logs that user doesn't want memoized. This is a better alternative to
       -- doing arbitrary IO from a (memoized) Haxl computation.

  , Env u w -> Maybe (DataCacheLookup w)
dataCacheFetchFallback :: !(Maybe (DataCacheLookup w))
       -- ^ Allows you to inject a DataCache lookup just before a dataFetch is
       -- dispatched. This is useful for injecting fetch results in testing.

#ifdef PROFILING
  , callGraphRef ::  Maybe (IORef CallGraph)
       -- ^ An edge list representing the current function call graph. The type
       -- is wrapped in a Maybe to avoid changing the existing callsites.

  , currFunction :: QualFunction
       -- ^ The most recent function call.
#endif
  }

data ProfileCurrent = ProfileCurrent
  { ProfileCurrent -> ProfileKey
profCurrentKey ::  {-# UNPACK #-} !ProfileKey
  , ProfileCurrent -> NonEmpty ProfileLabel
profLabelStack :: {-# UNPACK #-} !(NonEmpty ProfileLabel)
  }

type Caches u w = (HaxlDataCache u w, HaxlDataCache u w)

caches :: Env u w -> Caches u w
caches :: Env u w -> Caches u w
caches Env u w
env = (Env u w -> HaxlDataCache u w
forall u w. Env u w -> HaxlDataCache u w
dataCache Env u w
env, Env u w -> HaxlDataCache u w
forall u w. Env u w -> HaxlDataCache u w
memoCache Env u w
env)

getMaxCallId :: HaxlDataCache u w -> IO (Maybe Int)
getMaxCallId :: HaxlDataCache u w -> IO (Maybe CallId)
getMaxCallId HaxlDataCache u w
c = do
  [CallId]
callIds  <- [Either SomeException CallId] -> [CallId]
forall a b. [Either a b] -> [b]
rights ([Either SomeException CallId] -> [CallId])
-> ([(TypeRep, [Either SomeException CallId])]
    -> [Either SomeException CallId])
-> [(TypeRep, [Either SomeException CallId])]
-> [CallId]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((TypeRep, [Either SomeException CallId])
 -> [Either SomeException CallId])
-> [(TypeRep, [Either SomeException CallId])]
-> [Either SomeException CallId]
forall (t :: * -> *) a b. Foldable t => (a -> [b]) -> t a -> [b]
concatMap (TypeRep, [Either SomeException CallId])
-> [Either SomeException CallId]
forall a b. (a, b) -> b
snd ([(TypeRep, [Either SomeException CallId])] -> [CallId])
-> IO [(TypeRep, [Either SomeException CallId])] -> IO [CallId]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
              HaxlDataCache u w
-> (forall a. DataCacheItem u w a -> IO CallId)
-> IO [(TypeRep, [Either SomeException CallId])]
forall (res :: * -> *) ret.
DataCache res
-> (forall a. res a -> IO ret)
-> IO [(TypeRep, [Either SomeException ret])]
DataCache.readCache HaxlDataCache u w
c (\(DataCacheItem IVar u w a
_ CallId
i) -> CallId -> IO CallId
forall (m :: * -> *) a. Monad m => a -> m a
return CallId
i)
  case [CallId]
callIds of
    [] -> Maybe CallId -> IO (Maybe CallId)
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe CallId
forall a. Maybe a
Nothing
    [CallId]
vals -> Maybe CallId -> IO (Maybe CallId)
forall (m :: * -> *) a. Monad m => a -> m a
return (Maybe CallId -> IO (Maybe CallId))
-> Maybe CallId -> IO (Maybe CallId)
forall a b. (a -> b) -> a -> b
$ CallId -> Maybe CallId
forall a. a -> Maybe a
Just ([CallId] -> CallId
forall (t :: * -> *) a. (Foldable t, Ord a) => t a -> a
maximum [CallId]
vals)


-- | Initialize an environment with a 'StateStore', an input map, a
-- preexisting 'DataCache', and a seed for the random number generator.
initEnvWithData :: StateStore -> u -> Caches u w -> IO (Env u w)
initEnvWithData :: StateStore -> u -> Caches u w -> IO (Env u w)
initEnvWithData StateStore
states u
e (HaxlDataCache u w
dcache, HaxlDataCache u w
mcache) = do
  CallId
newCid <- CallId -> CallId -> CallId
forall a. Ord a => a -> a -> a
max (CallId -> CallId -> CallId) -> IO CallId -> IO (CallId -> CallId)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$>
    (CallId -> (CallId -> CallId) -> Maybe CallId -> CallId
forall b a. b -> (a -> b) -> Maybe a -> b
maybe CallId
0 (CallId -> CallId -> CallId
forall a. Num a => a -> a -> a
(+) CallId
1) (Maybe CallId -> CallId) -> IO (Maybe CallId) -> IO CallId
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HaxlDataCache u w -> IO (Maybe CallId)
forall u w. HaxlDataCache u w -> IO (Maybe CallId)
getMaxCallId HaxlDataCache u w
dcache) IO (CallId -> CallId) -> IO CallId -> IO CallId
forall (f :: * -> *) a b. Applicative f => f (a -> b) -> f a -> f b
<*>
    (CallId -> (CallId -> CallId) -> Maybe CallId -> CallId
forall b a. b -> (a -> b) -> Maybe a -> b
maybe CallId
0 (CallId -> CallId -> CallId
forall a. Num a => a -> a -> a
(+) CallId
1) (Maybe CallId -> CallId) -> IO (Maybe CallId) -> IO CallId
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HaxlDataCache u w -> IO (Maybe CallId)
forall u w. HaxlDataCache u w -> IO (Maybe CallId)
getMaxCallId HaxlDataCache u w
mcache)
  IORef CallId
ciref<- CallId -> IO (IORef CallId)
forall a. a -> IO (IORef a)
newIORef CallId
newCid
  IORef Stats
sref <- Stats -> IO (IORef Stats)
forall a. a -> IO (IORef a)
newIORef Stats
emptyStats
  IORef CallId
sbref <- CallId -> IO (IORef CallId)
forall a. a -> IO (IORef a)
newIORef CallId
0
  IORef Profile
pref <- Profile -> IO (IORef Profile)
forall a. a -> IO (IORef a)
newIORef Profile
emptyProfile
  IORef (RequestStore u)
rs <- RequestStore u -> IO (IORef (RequestStore u))
forall a. a -> IO (IORef a)
newIORef RequestStore u
forall u. RequestStore u
noRequests          -- RequestStore
  IORef (JobList u w)
rq <- JobList u w -> IO (IORef (JobList u w))
forall a. a -> IO (IORef a)
newIORef JobList u w
forall u w. JobList u w
JobNil              -- RunQueue
  IORef ReqCountMap
sr <- ReqCountMap -> IO (IORef ReqCountMap)
forall a. a -> IO (IORef a)
newIORef ReqCountMap
emptyReqCounts      -- SubmittedReqs
  TVar [CompleteReq u w]
comps <- [CompleteReq u w] -> IO (TVar [CompleteReq u w])
forall a. a -> IO (TVar a)
newTVarIO []              -- completion queue
  IORef (WriteTree w)
wl <- WriteTree w -> IO (IORef (WriteTree w))
forall a. a -> IO (IORef a)
newIORef WriteTree w
forall w. WriteTree w
NilWrites
  IORef (WriteTree w)
wlnm <- WriteTree w -> IO (IORef (WriteTree w))
forall a. a -> IO (IORef a)
newIORef WriteTree w
forall w. WriteTree w
NilWrites
  Env u w -> IO (Env u w)
forall (m :: * -> *) a. Monad m => a -> m a
return Env :: forall u w.
HaxlDataCache u w
-> HaxlDataCache u w
-> CallId
-> Flags
-> u
-> IORef Stats
-> IORef CallId
-> IORef CallId
-> ProfileCurrent
-> IORef Profile
-> StateStore
-> IORef (RequestStore u)
-> IORef (JobList u w)
-> IORef ReqCountMap
-> TVar [CompleteReq u w]
-> IORef (WriteTree w)
-> IORef (WriteTree w)
-> Maybe (DataCacheLookup w)
-> Env u w
Env
    { dataCache :: HaxlDataCache u w
dataCache = HaxlDataCache u w
dcache
    , memoCache :: HaxlDataCache u w
memoCache = HaxlDataCache u w
mcache
    , memoKey :: CallId
memoKey = (-CallId
1)
    , flags :: Flags
flags = Flags
defaultFlags
    , userEnv :: u
userEnv = u
e
    , states :: StateStore
states = StateStore
states
    , statsRef :: IORef Stats
statsRef = IORef Stats
sref
    , statsBatchIdRef :: IORef CallId
statsBatchIdRef = IORef CallId
sbref
    , profCurrent :: ProfileCurrent
profCurrent = ProfileKey -> NonEmpty ProfileLabel -> ProfileCurrent
ProfileCurrent ProfileKey
0 (NonEmpty ProfileLabel -> ProfileCurrent)
-> NonEmpty ProfileLabel -> ProfileCurrent
forall a b. (a -> b) -> a -> b
$ ProfileLabel
"MAIN" ProfileLabel -> [ProfileLabel] -> NonEmpty ProfileLabel
forall a. a -> [a] -> NonEmpty a
:| []
    , callIdRef :: IORef CallId
callIdRef = IORef CallId
ciref
    , profRef :: IORef Profile
profRef = IORef Profile
pref
    , reqStoreRef :: IORef (RequestStore u)
reqStoreRef = IORef (RequestStore u)
rs
    , runQueueRef :: IORef (JobList u w)
runQueueRef = IORef (JobList u w)
rq
    , submittedReqsRef :: IORef ReqCountMap
submittedReqsRef = IORef ReqCountMap
sr
    , completions :: TVar [CompleteReq u w]
completions = TVar [CompleteReq u w]
comps
    , writeLogsRef :: IORef (WriteTree w)
writeLogsRef = IORef (WriteTree w)
wl
    , writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRefNoMemo = IORef (WriteTree w)
wlnm
    , dataCacheFetchFallback :: Maybe (DataCacheLookup w)
dataCacheFetchFallback = Maybe (DataCacheLookup w)
forall a. Maybe a
Nothing
#ifdef PROFILING
    , callGraphRef = Nothing
    , currFunction = mainFunction
#endif
    }

-- | Initializes an environment with 'StateStore' and an input map.
initEnv :: StateStore -> u -> IO (Env u w)
initEnv :: StateStore -> u -> IO (Env u w)
initEnv StateStore
states u
e = do
  DataCache (DataCacheItem u w)
dcache <- IO (DataCache (DataCacheItem u w))
forall (res :: * -> *). IO (DataCache res)
emptyDataCache
  DataCache (DataCacheItem u w)
mcache <- IO (DataCache (DataCacheItem u w))
forall (res :: * -> *). IO (DataCache res)
emptyDataCache
  StateStore -> u -> Caches u w -> IO (Env u w)
forall u w. StateStore -> u -> Caches u w -> IO (Env u w)
initEnvWithData StateStore
states u
e (DataCache (DataCacheItem u w)
dcache, DataCache (DataCacheItem u w)
mcache)

-- | A new, empty environment.
emptyEnv :: u -> IO (Env u w)
emptyEnv :: u -> IO (Env u w)
emptyEnv = StateStore -> u -> IO (Env u w)
forall u w. StateStore -> u -> IO (Env u w)
initEnv StateStore
stateEmpty

-- | If you're using the env from a failed Haxl computation in a second Haxl
-- computation, it is recommended to sanitize the Env to remove all empty
-- IVars - especially if it's possible the first Haxl computation could've
-- been interrupted via an async exception. This is because if the Haxl
-- computation was interrupted by an exception, it's possible that there are
-- entries in the cache which are still blocked, while the results from
-- outgone fetches have been discarded.
sanitizeEnv :: Env u w -> IO (Env u w)
sanitizeEnv :: Env u w -> IO (Env u w)
sanitizeEnv env :: Env u w
env@Env{u
CallId
Maybe (DataCacheLookup w)
TVar [CompleteReq u w]
IORef CallId
IORef Profile
IORef Stats
IORef ReqCountMap
IORef (RequestStore u)
IORef (JobList u w)
IORef (WriteTree w)
HaxlDataCache u w
Flags
StateStore
ProfileCurrent
dataCacheFetchFallback :: Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRef :: IORef (WriteTree w)
completions :: TVar [CompleteReq u w]
submittedReqsRef :: IORef ReqCountMap
runQueueRef :: IORef (JobList u w)
reqStoreRef :: IORef (RequestStore u)
states :: StateStore
profRef :: IORef Profile
profCurrent :: ProfileCurrent
callIdRef :: IORef CallId
statsBatchIdRef :: IORef CallId
statsRef :: IORef Stats
userEnv :: u
flags :: Flags
memoKey :: CallId
memoCache :: HaxlDataCache u w
dataCache :: HaxlDataCache u w
dataCacheFetchFallback :: forall u w. Env u w -> Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef :: forall u w. Env u w -> IORef (WriteTree w)
completions :: forall u w. Env u w -> TVar [CompleteReq u w]
submittedReqsRef :: forall u w. Env u w -> IORef ReqCountMap
runQueueRef :: forall u w. Env u w -> IORef (JobList u w)
reqStoreRef :: forall u w. Env u w -> IORef (RequestStore u)
states :: forall u w. Env u w -> StateStore
profRef :: forall u w. Env u w -> IORef Profile
profCurrent :: forall u w. Env u w -> ProfileCurrent
callIdRef :: forall u w. Env u w -> IORef CallId
statsBatchIdRef :: forall u w. Env u w -> IORef CallId
statsRef :: forall u w. Env u w -> IORef Stats
userEnv :: forall u w. Env u w -> u
flags :: forall u w. Env u w -> Flags
memoKey :: forall u w. Env u w -> CallId
memoCache :: forall u w. Env u w -> HaxlDataCache u w
dataCache :: forall u w. Env u w -> HaxlDataCache u w
..} = do
  HaxlDataCache u w
sanitizedDC <- (forall a. DataCacheItem u w a -> IO Bool)
-> HaxlDataCache u w -> IO (HaxlDataCache u w)
forall (res :: * -> *).
(forall a. res a -> IO Bool) -> DataCache res -> IO (DataCache res)
DataCache.filter forall a. DataCacheItem u w a -> IO Bool
forall u w a. DataCacheItem u w a -> IO Bool
isIVarFull HaxlDataCache u w
dataCache
  HaxlDataCache u w
sanitizedMC <- (forall a. DataCacheItem u w a -> IO Bool)
-> HaxlDataCache u w -> IO (HaxlDataCache u w)
forall (res :: * -> *).
(forall a. res a -> IO Bool) -> DataCache res -> IO (DataCache res)
DataCache.filter forall a. DataCacheItem u w a -> IO Bool
forall u w a. DataCacheItem u w a -> IO Bool
isIVarFull HaxlDataCache u w
memoCache
  IORef (RequestStore u)
rs <- RequestStore u -> IO (IORef (RequestStore u))
forall a. a -> IO (IORef a)
newIORef RequestStore u
forall u. RequestStore u
noRequests          -- RequestStore
  IORef (JobList u w)
rq <- JobList u w -> IO (IORef (JobList u w))
forall a. a -> IO (IORef a)
newIORef JobList u w
forall u w. JobList u w
JobNil              -- RunQueue
  TVar [CompleteReq u w]
comps <- [CompleteReq u w] -> IO (TVar [CompleteReq u w])
forall a. a -> IO (TVar a)
newTVarIO []              -- completion queue
  IORef ReqCountMap
sr <- ReqCountMap -> IO (IORef ReqCountMap)
forall a. a -> IO (IORef a)
newIORef ReqCountMap
emptyReqCounts      -- SubmittedReqs
  Env u w -> IO (Env u w)
forall (m :: * -> *) a. Monad m => a -> m a
return Env u w
env
    { dataCache :: HaxlDataCache u w
dataCache = HaxlDataCache u w
sanitizedDC
    , memoCache :: HaxlDataCache u w
memoCache = HaxlDataCache u w
sanitizedMC
    , reqStoreRef :: IORef (RequestStore u)
reqStoreRef = IORef (RequestStore u)
rs
    , runQueueRef :: IORef (JobList u w)
runQueueRef = IORef (JobList u w)
rq
    , completions :: TVar [CompleteReq u w]
completions = TVar [CompleteReq u w]
comps
    , submittedReqsRef :: IORef ReqCountMap
submittedReqsRef = IORef ReqCountMap
sr
    }
  where
  isIVarFull :: DataCacheItem u w a -> IO Bool
isIVarFull (DataCacheItem IVar{IORef (IVarContents u w a)
ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef :: IORef (IVarContents u w a)
..} CallId
_) = do
    IVarContents u w a
ivarContents <- IORef (IVarContents u w a) -> IO (IVarContents u w a)
forall a. IORef a -> IO a
readIORef IORef (IVarContents u w a)
ivarRef
    case IVarContents u w a
ivarContents of
      IVarFull ResultVal a w
_ -> Bool -> IO Bool
forall (m :: * -> *) a. Monad m => a -> m a
return Bool
True
      IVarContents u w a
_ -> Bool -> IO Bool
forall (m :: * -> *) a. Monad m => a -> m a
return Bool
False

-- -----------------------------------------------------------------------------
-- WriteTree

-- | A tree of writes done during a Haxl computation. We could use a simple
-- list, but this allows us to avoid multiple mappends when concatenating
-- writes from two haxl computations.
--
-- Users should try to treat this data type as opaque, and prefer
-- to use @flattenWT@ to get a simple list of writes from a @WriteTree@.
data WriteTree w
  = NilWrites
  | SomeWrite w
  | MergeWrites (WriteTree w) (WriteTree w)
  deriving (CallId -> WriteTree w -> ShowS
[WriteTree w] -> ShowS
WriteTree w -> String
(CallId -> WriteTree w -> ShowS)
-> (WriteTree w -> String)
-> ([WriteTree w] -> ShowS)
-> Show (WriteTree w)
forall w. Show w => CallId -> WriteTree w -> ShowS
forall w. Show w => [WriteTree w] -> ShowS
forall w. Show w => WriteTree w -> String
forall a.
(CallId -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
showList :: [WriteTree w] -> ShowS
$cshowList :: forall w. Show w => [WriteTree w] -> ShowS
show :: WriteTree w -> String
$cshow :: forall w. Show w => WriteTree w -> String
showsPrec :: CallId -> WriteTree w -> ShowS
$cshowsPrec :: forall w. Show w => CallId -> WriteTree w -> ShowS
Show)

appendWTs :: WriteTree w -> WriteTree w -> WriteTree w
appendWTs :: WriteTree w -> WriteTree w -> WriteTree w
appendWTs WriteTree w
NilWrites WriteTree w
w = WriteTree w
w
appendWTs WriteTree w
w WriteTree w
NilWrites = WriteTree w
w
appendWTs WriteTree w
w1 WriteTree w
w2 = WriteTree w -> WriteTree w -> WriteTree w
forall w. WriteTree w -> WriteTree w -> WriteTree w
MergeWrites WriteTree w
w1 WriteTree w
w2

-- This function must be called at the end of the Haxl computation to get
-- a list of writes.
-- Haxl provides no guarantees on the order of the returned logs.
flattenWT :: WriteTree w -> [w]
flattenWT :: WriteTree w -> [w]
flattenWT = [w] -> WriteTree w -> [w]
forall a. [a] -> WriteTree a -> [a]
go []
  where
    go :: [a] -> WriteTree a -> [a]
go ![a]
ws WriteTree a
NilWrites = [a]
ws
    go ![a]
ws (SomeWrite a
w) = a
w a -> [a] -> [a]
forall a. a -> [a] -> [a]
: [a]
ws
    go ![a]
ws (MergeWrites WriteTree a
w1 WriteTree a
w2) = [a] -> WriteTree a -> [a]
go ([a] -> WriteTree a -> [a]
go [a]
ws WriteTree a
w2) WriteTree a
w1

-- This is a convenience wrapper over modifyIORef, which only modifies
-- writeLogsRef IORef, for non NilWrites.
mbModifyWLRef :: WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef :: WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef WriteTree w
NilWrites IORef (WriteTree w)
_ = () -> IO ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()
mbModifyWLRef !WriteTree w
wt IORef (WriteTree w)
ref = IORef (WriteTree w) -> (WriteTree w -> WriteTree w) -> IO ()
forall a. IORef a -> (a -> a) -> IO ()
modifyIORef' IORef (WriteTree w)
ref (WriteTree w -> WriteTree w -> WriteTree w
forall w. WriteTree w -> WriteTree w -> WriteTree w
`appendWTs` WriteTree w
wt)

mapWriteTree :: (w -> w) -> WriteTree w -> WriteTree w
mapWriteTree :: (w -> w) -> WriteTree w -> WriteTree w
mapWriteTree w -> w
_ WriteTree w
NilWrites = WriteTree w
forall w. WriteTree w
NilWrites
mapWriteTree w -> w
f (SomeWrite w
w) = w -> WriteTree w
forall w. w -> WriteTree w
SomeWrite (w -> w
f w
w)
mapWriteTree w -> w
f (MergeWrites WriteTree w
wt1 WriteTree w
wt2) =
  WriteTree w -> WriteTree w -> WriteTree w
forall w. WriteTree w -> WriteTree w -> WriteTree w
MergeWrites ((w -> w) -> WriteTree w -> WriteTree w
forall w. (w -> w) -> WriteTree w -> WriteTree w
mapWriteTree w -> w
f WriteTree w
wt1) ((w -> w) -> WriteTree w -> WriteTree w
forall w. (w -> w) -> WriteTree w -> WriteTree w
mapWriteTree w -> w
f WriteTree w
wt2)

-- -----------------------------------------------------------------------------
-- | The Haxl monad, which does several things:
--
--  * It is a reader monad for 'Env', which contains the current state
--    of the scheduler, including unfetched requests and the run queue
--    of computations.
--
--  * It is a writer monad for 'WriteTree'. These can be used to do
--    arbitrary "logs" from any Haxl computation. These are better than
--    doing arbitrary IO from a Haxl computation as these writes also get
--    memoized if the Haxl computation associated with them is memoized.
--    Now if this memoized computation is run again, you'll get the writes
--    twice.

--  * It is a concurrency, or resumption, monad. A computation may run
--    partially and return 'Blocked', in which case the framework should
--    perform the outstanding requests in the 'RequestStore', and then
--    resume the computation.
--
--  * The Applicative combinator '<*>' explores /both/ branches in the
--    event that the left branch is 'Blocked', so that we can collect
--    multiple requests and submit them as a batch.
--
--  * It contains IO, so that we can perform real data fetching.
--
newtype GenHaxl u w a = GenHaxl
  { GenHaxl u w a -> Env u w -> IO (Result u w a)
unHaxl :: Env u w -> IO (Result u w a) }

tellWrite :: w -> GenHaxl u w ()
tellWrite :: w -> GenHaxl u w ()
tellWrite = WriteTree w -> GenHaxl u w ()
forall w u. WriteTree w -> GenHaxl u w ()
write (WriteTree w -> GenHaxl u w ())
-> (w -> WriteTree w) -> w -> GenHaxl u w ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. w -> WriteTree w
forall w. w -> WriteTree w
SomeWrite

write :: WriteTree w -> GenHaxl u w ()
write :: WriteTree w -> GenHaxl u w ()
write WriteTree w
wt = (Env u w -> IO (Result u w ())) -> GenHaxl u w ()
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w ())) -> GenHaxl u w ())
-> (Env u w -> IO (Result u w ())) -> GenHaxl u w ()
forall a b. (a -> b) -> a -> b
$ \Env{u
CallId
Maybe (DataCacheLookup w)
TVar [CompleteReq u w]
IORef CallId
IORef Profile
IORef Stats
IORef ReqCountMap
IORef (RequestStore u)
IORef (JobList u w)
IORef (WriteTree w)
HaxlDataCache u w
Flags
StateStore
ProfileCurrent
dataCacheFetchFallback :: Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRef :: IORef (WriteTree w)
completions :: TVar [CompleteReq u w]
submittedReqsRef :: IORef ReqCountMap
runQueueRef :: IORef (JobList u w)
reqStoreRef :: IORef (RequestStore u)
states :: StateStore
profRef :: IORef Profile
profCurrent :: ProfileCurrent
callIdRef :: IORef CallId
statsBatchIdRef :: IORef CallId
statsRef :: IORef Stats
userEnv :: u
flags :: Flags
memoKey :: CallId
memoCache :: HaxlDataCache u w
dataCache :: HaxlDataCache u w
dataCacheFetchFallback :: forall u w. Env u w -> Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef :: forall u w. Env u w -> IORef (WriteTree w)
completions :: forall u w. Env u w -> TVar [CompleteReq u w]
submittedReqsRef :: forall u w. Env u w -> IORef ReqCountMap
runQueueRef :: forall u w. Env u w -> IORef (JobList u w)
reqStoreRef :: forall u w. Env u w -> IORef (RequestStore u)
states :: forall u w. Env u w -> StateStore
profRef :: forall u w. Env u w -> IORef Profile
profCurrent :: forall u w. Env u w -> ProfileCurrent
callIdRef :: forall u w. Env u w -> IORef CallId
statsBatchIdRef :: forall u w. Env u w -> IORef CallId
statsRef :: forall u w. Env u w -> IORef Stats
userEnv :: forall u w. Env u w -> u
flags :: forall u w. Env u w -> Flags
memoKey :: forall u w. Env u w -> CallId
memoCache :: forall u w. Env u w -> HaxlDataCache u w
dataCache :: forall u w. Env u w -> HaxlDataCache u w
..} -> do
  WriteTree w -> IORef (WriteTree w) -> IO ()
forall w. WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef WriteTree w
wt IORef (WriteTree w)
writeLogsRef
  Result u w () -> IO (Result u w ())
forall (m :: * -> *) a. Monad m => a -> m a
return (Result u w () -> IO (Result u w ()))
-> Result u w () -> IO (Result u w ())
forall a b. (a -> b) -> a -> b
$ () -> Result u w ()
forall u w a. a -> Result u w a
Done ()

tellWriteNoMemo :: w -> GenHaxl u w ()
tellWriteNoMemo :: w -> GenHaxl u w ()
tellWriteNoMemo = WriteTree w -> GenHaxl u w ()
forall w u. WriteTree w -> GenHaxl u w ()
writeNoMemo (WriteTree w -> GenHaxl u w ())
-> (w -> WriteTree w) -> w -> GenHaxl u w ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. w -> WriteTree w
forall w. w -> WriteTree w
SomeWrite

writeNoMemo :: WriteTree w -> GenHaxl u w ()
writeNoMemo :: WriteTree w -> GenHaxl u w ()
writeNoMemo WriteTree w
wt = (Env u w -> IO (Result u w ())) -> GenHaxl u w ()
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w ())) -> GenHaxl u w ())
-> (Env u w -> IO (Result u w ())) -> GenHaxl u w ()
forall a b. (a -> b) -> a -> b
$ \Env{u
CallId
Maybe (DataCacheLookup w)
TVar [CompleteReq u w]
IORef CallId
IORef Profile
IORef Stats
IORef ReqCountMap
IORef (RequestStore u)
IORef (JobList u w)
IORef (WriteTree w)
HaxlDataCache u w
Flags
StateStore
ProfileCurrent
dataCacheFetchFallback :: Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRef :: IORef (WriteTree w)
completions :: TVar [CompleteReq u w]
submittedReqsRef :: IORef ReqCountMap
runQueueRef :: IORef (JobList u w)
reqStoreRef :: IORef (RequestStore u)
states :: StateStore
profRef :: IORef Profile
profCurrent :: ProfileCurrent
callIdRef :: IORef CallId
statsBatchIdRef :: IORef CallId
statsRef :: IORef Stats
userEnv :: u
flags :: Flags
memoKey :: CallId
memoCache :: HaxlDataCache u w
dataCache :: HaxlDataCache u w
dataCacheFetchFallback :: forall u w. Env u w -> Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef :: forall u w. Env u w -> IORef (WriteTree w)
completions :: forall u w. Env u w -> TVar [CompleteReq u w]
submittedReqsRef :: forall u w. Env u w -> IORef ReqCountMap
runQueueRef :: forall u w. Env u w -> IORef (JobList u w)
reqStoreRef :: forall u w. Env u w -> IORef (RequestStore u)
states :: forall u w. Env u w -> StateStore
profRef :: forall u w. Env u w -> IORef Profile
profCurrent :: forall u w. Env u w -> ProfileCurrent
callIdRef :: forall u w. Env u w -> IORef CallId
statsBatchIdRef :: forall u w. Env u w -> IORef CallId
statsRef :: forall u w. Env u w -> IORef Stats
userEnv :: forall u w. Env u w -> u
flags :: forall u w. Env u w -> Flags
memoKey :: forall u w. Env u w -> CallId
memoCache :: forall u w. Env u w -> HaxlDataCache u w
dataCache :: forall u w. Env u w -> HaxlDataCache u w
..} -> do
  WriteTree w -> IORef (WriteTree w) -> IO ()
forall w. WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef WriteTree w
wt IORef (WriteTree w)
writeLogsRefNoMemo
  Result u w () -> IO (Result u w ())
forall (m :: * -> *) a. Monad m => a -> m a
return (Result u w () -> IO (Result u w ()))
-> Result u w () -> IO (Result u w ())
forall a b. (a -> b) -> a -> b
$ () -> Result u w ()
forall u w a. a -> Result u w a
Done ()


instance IsString a => IsString (GenHaxl u w a) where
  fromString :: String -> GenHaxl u w a
fromString String
s = a -> GenHaxl u w a
forall (m :: * -> *) a. Monad m => a -> m a
return (String -> a
forall a. IsString a => String -> a
fromString String
s)

-- -----------------------------------------------------------------------------
-- JobList

-- | A list of computations together with the IVar into which they
-- should put their result.
--
-- This could be an ordinary list, but the optimised representation
-- saves space and time.
--
data JobList u w
 = JobNil
 | forall a . JobCons
     (Env u w)          -- See Note [make withEnv work] below.
     (GenHaxl u w a)
     {-# UNPACK #-} !(IVar u w a)
     (JobList u w)

-- Note [make withEnv work]
--
-- The withEnv operation supplies a new Env for the scope of a GenHaxl
-- computation.  The problem is that the computation might be split
-- into pieces and put onto various JobLists, so we have to be sure to
-- use the correct Env when we execute the pieces. Furthermore, if one
-- of these pieces blocks and gets run again later, we must ensure to
-- restart it with the correct Env.  So we stash the Env along with
-- the continuation in the JobList.

appendJobList :: JobList u w -> JobList u w -> JobList u w
appendJobList :: JobList u w -> JobList u w -> JobList u w
appendJobList JobList u w
JobNil JobList u w
c = JobList u w
c
appendJobList JobList u w
c JobList u w
JobNil = JobList u w
c
appendJobList (JobCons Env u w
a GenHaxl u w a
b IVar u w a
c JobList u w
d) JobList u w
e = Env u w
-> GenHaxl u w a -> IVar u w a -> JobList u w -> JobList u w
forall u w a.
Env u w
-> GenHaxl u w a -> IVar u w a -> JobList u w -> JobList u w
JobCons Env u w
a GenHaxl u w a
b IVar u w a
c (JobList u w -> JobList u w) -> JobList u w -> JobList u w
forall a b. (a -> b) -> a -> b
$! JobList u w -> JobList u w -> JobList u w
forall u w. JobList u w -> JobList u w -> JobList u w
appendJobList JobList u w
d JobList u w
e

lengthJobList :: JobList u w -> Int
lengthJobList :: JobList u w -> CallId
lengthJobList JobList u w
JobNil = CallId
0
lengthJobList (JobCons Env u w
_ GenHaxl u w a
_ IVar u w a
_ JobList u w
j) = CallId
1 CallId -> CallId -> CallId
forall a. Num a => a -> a -> a
+ JobList u w -> CallId
forall u w. JobList u w -> CallId
lengthJobList JobList u w
j


-- -----------------------------------------------------------------------------
-- IVar

-- | A synchronisation point.  It either contains a value, or a list
-- of computations waiting for the value.
#ifdef PROFILING
data IVar u w a = IVar
  { ivarRef :: {-# UNPACK #-} !(IORef (IVarContents u w a))
  , ivarCCS :: {-# UNPACK #-} !(Ptr CostCentreStack)
#else
newtype IVar u w a = IVar
  { IVar u w a -> IORef (IVarContents u w a)
ivarRef :: IORef (IVarContents u w a)
#endif
  }

data IVarContents u w a
  = IVarFull (ResultVal a w)
  | IVarEmpty (JobList u w)
    -- morally this is a list of @a -> GenHaxl u w ()@, but instead of
    -- using a function, each computation begins with `getIVar` to grab
    -- the value it is waiting for.  This is less type safe but a little
    -- faster (benchmarked with tests/MonadBench.hs).

newIVar :: IO (IVar u w a)
newIVar :: IO (IVar u w a)
newIVar = do
  IORef (IVarContents u w a)
ivarRef <- IVarContents u w a -> IO (IORef (IVarContents u w a))
forall a. a -> IO (IORef a)
newIORef (JobList u w -> IVarContents u w a
forall u w a. JobList u w -> IVarContents u w a
IVarEmpty JobList u w
forall u w. JobList u w
JobNil)
#ifdef PROFILING
  ivarCCS <- getCurrentCCS ivarRef
#endif
  IVar u w a -> IO (IVar u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return IVar :: forall u w a. IORef (IVarContents u w a) -> IVar u w a
IVar{IORef (IVarContents u w a)
ivarRef :: IORef (IVarContents u w a)
ivarRef :: IORef (IVarContents u w a)
..}

newFullIVar :: ResultVal a w -> IO (IVar u w a)
newFullIVar :: ResultVal a w -> IO (IVar u w a)
newFullIVar ResultVal a w
r = do
  IORef (IVarContents u w a)
ivarRef <- IVarContents u w a -> IO (IORef (IVarContents u w a))
forall a. a -> IO (IORef a)
newIORef (ResultVal a w -> IVarContents u w a
forall u w a. ResultVal a w -> IVarContents u w a
IVarFull ResultVal a w
r)
#ifdef PROFILING
  ivarCCS <- getCurrentCCS ivarRef
#endif
  IVar u w a -> IO (IVar u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return IVar :: forall u w a. IORef (IVarContents u w a) -> IVar u w a
IVar{IORef (IVarContents u w a)
ivarRef :: IORef (IVarContents u w a)
ivarRef :: IORef (IVarContents u w a)
..}

withCurrentCCS :: IVar u w a -> IO (IVar u w a)
#ifdef PROFILING
withCurrentCCS ivar = do
  ccs <- getCurrentCCS ivar
  return ivar{ivarCCS = ccs}
#else
withCurrentCCS :: IVar u w a -> IO (IVar u w a)
withCurrentCCS = IVar u w a -> IO (IVar u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return
#endif

getIVar :: IVar u w a -> GenHaxl u w a
getIVar :: IVar u w a -> GenHaxl u w a
getIVar i :: IVar u w a
i@IVar{ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef = !IORef (IVarContents u w a)
ref} = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
  IVarContents u w a
e <- IORef (IVarContents u w a) -> IO (IVarContents u w a)
forall a. IORef a -> IO a
readIORef IORef (IVarContents u w a)
ref
  case IVarContents u w a
e of
    IVarFull (Ok a
a WriteTree w
_wt) -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
    IVarFull (ThrowHaxl SomeException
e WriteTree w
_wt) -> Env u w -> IVar u w a -> SomeException -> IO (Result u w a)
forall e u w a b.
Exception e =>
Env u w -> IVar u w a -> e -> IO (Result u w b)
raiseFromIVar Env u w
env IVar u w a
i SomeException
e
    IVarFull (ThrowIO SomeException
e) -> SomeException -> IO (Result u w a)
forall e a. Exception e => e -> IO a
throwIO SomeException
e
    IVarEmpty JobList u w
_ -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w a -> Cont u w a -> Result u w a
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w a
i (IVar u w a -> Cont u w a
forall u w a. IVar u w a -> Cont u w a
Return IVar u w a
i))

-- Just a specialised version of getIVar, for efficiency in <*>
getIVarApply :: IVar u w (a -> b) -> a -> GenHaxl u w b
getIVarApply :: IVar u w (a -> b) -> a -> GenHaxl u w b
getIVarApply i :: IVar u w (a -> b)
i@IVar{ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef = !IORef (IVarContents u w (a -> b))
ref} a
a = (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w b)) -> GenHaxl u w b)
-> (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
  IVarContents u w (a -> b)
e <- IORef (IVarContents u w (a -> b)) -> IO (IVarContents u w (a -> b))
forall a. IORef a -> IO a
readIORef IORef (IVarContents u w (a -> b))
ref
  case IVarContents u w (a -> b)
e of
    IVarFull (Ok a -> b
f WriteTree w
_wt) -> Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (b -> Result u w b
forall u w a. a -> Result u w a
Done (a -> b
f a
a))
    IVarFull (ThrowHaxl SomeException
e WriteTree w
_wt) -> Env u w -> IVar u w (a -> b) -> SomeException -> IO (Result u w b)
forall e u w a b.
Exception e =>
Env u w -> IVar u w a -> e -> IO (Result u w b)
raiseFromIVar Env u w
env IVar u w (a -> b)
i SomeException
e
    IVarFull (ThrowIO SomeException
e) -> SomeException -> IO (Result u w b)
forall e a. Exception e => e -> IO a
throwIO SomeException
e
    IVarEmpty JobList u w
_ ->
      Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w (a -> b) -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w (a -> b)
i (GenHaxl u w b -> Cont u w b
forall u w a. GenHaxl u w a -> Cont u w a
Cont (IVar u w (a -> b) -> a -> GenHaxl u w b
forall u w a b. IVar u w (a -> b) -> a -> GenHaxl u w b
getIVarApply IVar u w (a -> b)
i a
a)))

-- Another specialised version of getIVar, for efficiency in cachedComputation
getIVarWithWrites :: IVar u w a -> GenHaxl u w a
getIVarWithWrites :: IVar u w a -> GenHaxl u w a
getIVarWithWrites i :: IVar u w a
i@IVar{ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef = !IORef (IVarContents u w a)
ref} = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \env :: Env u w
env@Env{u
CallId
Maybe (DataCacheLookup w)
TVar [CompleteReq u w]
IORef CallId
IORef Profile
IORef Stats
IORef ReqCountMap
IORef (RequestStore u)
IORef (JobList u w)
IORef (WriteTree w)
HaxlDataCache u w
Flags
StateStore
ProfileCurrent
dataCacheFetchFallback :: Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRef :: IORef (WriteTree w)
completions :: TVar [CompleteReq u w]
submittedReqsRef :: IORef ReqCountMap
runQueueRef :: IORef (JobList u w)
reqStoreRef :: IORef (RequestStore u)
states :: StateStore
profRef :: IORef Profile
profCurrent :: ProfileCurrent
callIdRef :: IORef CallId
statsBatchIdRef :: IORef CallId
statsRef :: IORef Stats
userEnv :: u
flags :: Flags
memoKey :: CallId
memoCache :: HaxlDataCache u w
dataCache :: HaxlDataCache u w
dataCacheFetchFallback :: forall u w. Env u w -> Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef :: forall u w. Env u w -> IORef (WriteTree w)
completions :: forall u w. Env u w -> TVar [CompleteReq u w]
submittedReqsRef :: forall u w. Env u w -> IORef ReqCountMap
runQueueRef :: forall u w. Env u w -> IORef (JobList u w)
reqStoreRef :: forall u w. Env u w -> IORef (RequestStore u)
states :: forall u w. Env u w -> StateStore
profRef :: forall u w. Env u w -> IORef Profile
profCurrent :: forall u w. Env u w -> ProfileCurrent
callIdRef :: forall u w. Env u w -> IORef CallId
statsBatchIdRef :: forall u w. Env u w -> IORef CallId
statsRef :: forall u w. Env u w -> IORef Stats
userEnv :: forall u w. Env u w -> u
flags :: forall u w. Env u w -> Flags
memoKey :: forall u w. Env u w -> CallId
memoCache :: forall u w. Env u w -> HaxlDataCache u w
dataCache :: forall u w. Env u w -> HaxlDataCache u w
..} -> do
  IVarContents u w a
e <- IORef (IVarContents u w a) -> IO (IVarContents u w a)
forall a. IORef a -> IO a
readIORef IORef (IVarContents u w a)
ref
  case IVarContents u w a
e of
    IVarFull (Ok a
a WriteTree w
wt) -> do
      WriteTree w -> IORef (WriteTree w) -> IO ()
forall w. WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef WriteTree w
wt IORef (WriteTree w)
writeLogsRef
      Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
    IVarFull (ThrowHaxl SomeException
e WriteTree w
wt) -> do
      WriteTree w -> IORef (WriteTree w) -> IO ()
forall w. WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef WriteTree w
wt IORef (WriteTree w)
writeLogsRef
      Env u w -> IVar u w a -> SomeException -> IO (Result u w a)
forall e u w a b.
Exception e =>
Env u w -> IVar u w a -> e -> IO (Result u w b)
raiseFromIVar Env u w
env IVar u w a
i SomeException
e
    IVarFull (ThrowIO SomeException
e) -> SomeException -> IO (Result u w a)
forall e a. Exception e => e -> IO a
throwIO SomeException
e
    IVarEmpty JobList u w
_ ->
      Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w a -> Cont u w a -> Result u w a
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w a
i (GenHaxl u w a -> Cont u w a
forall u w a. GenHaxl u w a -> Cont u w a
Cont (IVar u w a -> GenHaxl u w a
forall u w a. IVar u w a -> GenHaxl u w a
getIVarWithWrites IVar u w a
i)))

putIVar :: IVar u w a -> ResultVal a w -> Env u w -> IO ()
putIVar :: IVar u w a -> ResultVal a w -> Env u w -> IO ()
putIVar IVar{ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef = !IORef (IVarContents u w a)
ref} ResultVal a w
a Env{u
CallId
Maybe (DataCacheLookup w)
TVar [CompleteReq u w]
IORef CallId
IORef Profile
IORef Stats
IORef ReqCountMap
IORef (RequestStore u)
IORef (JobList u w)
IORef (WriteTree w)
HaxlDataCache u w
Flags
StateStore
ProfileCurrent
dataCacheFetchFallback :: Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRef :: IORef (WriteTree w)
completions :: TVar [CompleteReq u w]
submittedReqsRef :: IORef ReqCountMap
runQueueRef :: IORef (JobList u w)
reqStoreRef :: IORef (RequestStore u)
states :: StateStore
profRef :: IORef Profile
profCurrent :: ProfileCurrent
callIdRef :: IORef CallId
statsBatchIdRef :: IORef CallId
statsRef :: IORef Stats
userEnv :: u
flags :: Flags
memoKey :: CallId
memoCache :: HaxlDataCache u w
dataCache :: HaxlDataCache u w
dataCacheFetchFallback :: forall u w. Env u w -> Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef :: forall u w. Env u w -> IORef (WriteTree w)
completions :: forall u w. Env u w -> TVar [CompleteReq u w]
submittedReqsRef :: forall u w. Env u w -> IORef ReqCountMap
runQueueRef :: forall u w. Env u w -> IORef (JobList u w)
reqStoreRef :: forall u w. Env u w -> IORef (RequestStore u)
states :: forall u w. Env u w -> StateStore
profRef :: forall u w. Env u w -> IORef Profile
profCurrent :: forall u w. Env u w -> ProfileCurrent
callIdRef :: forall u w. Env u w -> IORef CallId
statsBatchIdRef :: forall u w. Env u w -> IORef CallId
statsRef :: forall u w. Env u w -> IORef Stats
userEnv :: forall u w. Env u w -> u
flags :: forall u w. Env u w -> Flags
memoKey :: forall u w. Env u w -> CallId
memoCache :: forall u w. Env u w -> HaxlDataCache u w
dataCache :: forall u w. Env u w -> HaxlDataCache u w
..} = do
  IVarContents u w a
e <- IORef (IVarContents u w a) -> IO (IVarContents u w a)
forall a. IORef a -> IO a
readIORef IORef (IVarContents u w a)
ref
  case IVarContents u w a
e of
    IVarEmpty JobList u w
jobs -> do
      IORef (IVarContents u w a) -> IVarContents u w a -> IO ()
forall a. IORef a -> a -> IO ()
writeIORef IORef (IVarContents u w a)
ref (ResultVal a w -> IVarContents u w a
forall u w a. ResultVal a w -> IVarContents u w a
IVarFull ResultVal a w
a)
      IORef (JobList u w) -> (JobList u w -> JobList u w) -> IO ()
forall a. IORef a -> (a -> a) -> IO ()
modifyIORef' IORef (JobList u w)
runQueueRef (JobList u w -> JobList u w -> JobList u w
forall u w. JobList u w -> JobList u w -> JobList u w
appendJobList JobList u w
jobs)
      -- An IVar is typically only meant to be written to once
      -- so it would make sense to throw an error here. But there
      -- are legitimate use-cases for writing several times.
      -- (See Haxl.Core.Parallel)
    IVarFull{} -> () -> IO ()
forall (m :: * -> *) a. Monad m => a -> m a
return ()

{-# INLINE addJob #-}
addJob :: Env u w -> GenHaxl u w b -> IVar u w b -> IVar u w a -> IO ()
addJob :: Env u w -> GenHaxl u w b -> IVar u w b -> IVar u w a -> IO ()
addJob Env u w
env !GenHaxl u w b
haxl !IVar u w b
resultIVar IVar{ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef = !IORef (IVarContents u w a)
ref} =
  IORef (IVarContents u w a)
-> (IVarContents u w a -> IVarContents u w a) -> IO ()
forall a. IORef a -> (a -> a) -> IO ()
modifyIORef' IORef (IVarContents u w a)
ref ((IVarContents u w a -> IVarContents u w a) -> IO ())
-> (IVarContents u w a -> IVarContents u w a) -> IO ()
forall a b. (a -> b) -> a -> b
$ \IVarContents u w a
contents ->
    case IVarContents u w a
contents of
      IVarEmpty JobList u w
list -> JobList u w -> IVarContents u w a
forall u w a. JobList u w -> IVarContents u w a
IVarEmpty (Env u w
-> GenHaxl u w b -> IVar u w b -> JobList u w -> JobList u w
forall u w a.
Env u w
-> GenHaxl u w a -> IVar u w a -> JobList u w -> JobList u w
JobCons Env u w
env GenHaxl u w b
haxl IVar u w b
resultIVar JobList u w
list)
      IVarContents u w a
_ -> IVarContents u w a
forall a. a
addJobPanic

addJobPanic :: forall a . a
addJobPanic :: a
addJobPanic = String -> a
forall a. HasCallStack => String -> a
error String
"addJob: not empty"


-- -----------------------------------------------------------------------------
-- ResultVal

-- | The contents of a full IVar.  We have to distinguish exceptions
-- thrown in the IO monad from exceptions thrown in the Haxl monad, so
-- that when the result is fetched using getIVar, we can throw the
-- exception in the right way.
data ResultVal a w
  = Ok a (WriteTree w)
  | ThrowHaxl SomeException (WriteTree w)
  | ThrowIO SomeException
    -- we get no write logs when an IO exception occurs

done :: Env u w -> ResultVal a w -> IO (Result u w a)
done :: Env u w -> ResultVal a w -> IO (Result u w a)
done Env u w
_ (Ok a
a WriteTree w
_) = Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
done Env u w
env (ThrowHaxl SomeException
e WriteTree w
_) = Env u w -> SomeException -> IO (Result u w a)
forall e u w a. Exception e => Env u w -> e -> IO (Result u w a)
raise Env u w
env SomeException
e
done Env u w
_ (ThrowIO SomeException
e) = SomeException -> IO (Result u w a)
forall e a. Exception e => e -> IO a
throwIO SomeException
e

eitherToResultThrowIO :: Either SomeException a -> ResultVal a w
eitherToResultThrowIO :: Either SomeException a -> ResultVal a w
eitherToResultThrowIO (Right a
a) = a -> WriteTree w -> ResultVal a w
forall a w. a -> WriteTree w -> ResultVal a w
Ok a
a WriteTree w
forall w. WriteTree w
NilWrites
eitherToResultThrowIO (Left SomeException
e)
  | Just HaxlException{} <- SomeException -> Maybe HaxlException
forall e. Exception e => SomeException -> Maybe e
fromException SomeException
e = SomeException -> WriteTree w -> ResultVal a w
forall a w. SomeException -> WriteTree w -> ResultVal a w
ThrowHaxl SomeException
e WriteTree w
forall w. WriteTree w
NilWrites
  | Bool
otherwise = SomeException -> ResultVal a w
forall a w. SomeException -> ResultVal a w
ThrowIO SomeException
e

eitherToResult :: Either SomeException a -> ResultVal a w
eitherToResult :: Either SomeException a -> ResultVal a w
eitherToResult (Right a
a) = a -> WriteTree w -> ResultVal a w
forall a w. a -> WriteTree w -> ResultVal a w
Ok a
a WriteTree w
forall w. WriteTree w
NilWrites
eitherToResult (Left SomeException
e) = SomeException -> WriteTree w -> ResultVal a w
forall a w. SomeException -> WriteTree w -> ResultVal a w
ThrowHaxl SomeException
e WriteTree w
forall w. WriteTree w
NilWrites


-- -----------------------------------------------------------------------------
-- CompleteReq

-- | A completed request from a data source, containing the result,
-- and the 'IVar' representing the blocked computations.  The job of a
-- data source is just to add these to a queue ('completions') using
-- 'putResult'; the scheduler collects them from the queue and unblocks
-- the relevant computations.
data CompleteReq u w
  = forall a . CompleteReq
      (ResultVal a w)
      !(IVar u w a)  -- IVar because the result is cached
      {-# UNPACK #-} !Int64 -- see Note [tracking allocation in child threads]


{- Note [tracking allocation in child threads]

For a BackgroundFetch, we might be doing some of the work in a
separate thread, but we want to make sure that the parent thread gets
charged for the allocation, so that allocation limits still work.

The design is a bit tricky here.  We want to track the allocation
accurately but without adding much overhead.

The best way to propagate the allocation back from the child thread is
through putResult.  If we had some other method, we would also need a
way to synchronise it with the main runHaxl loop; the advantage of
putResult is that this is already a synchronisation method, because
runHaxl is waiting for the result of the dataFetch.

We need a special version of putResult for child threads
(putResultFromChildThread), because we don't want to propagate any
allocation from the runHaxl thread back to itself and count it twice.

We also want to capture the allocation as late as possible, so that we
count everything.  For that reason, we pass a Bool down from putResult
into the function in the ResultVar, and it reads the allocation
counter as the last thing before adding the result to the completions
TVar.

The other problem to consider is how to capture the allocation when
the child thread is doing multiple putResults.  Our solution here is
to ensure that the *last* one is a putResultFromChildThread, so it
captures all the allocation from everything leading up to it.

Why not reset the counter each time, so we could do multiple
putResultFromChildThreads?  Because the child thread might be using an
allocation limit itself, and changing the counter would mess it up.
-}

-- -----------------------------------------------------------------------------
-- Result

-- | The result of a computation is either 'Done' with a value, 'Throw'
-- with an exception, or 'Blocked' on the result of a data fetch with
-- a continuation.
data Result u w a
  = Done a
  | Throw SomeException
  | forall b . Blocked
      {-# UNPACK #-} !(IVar u w b)
      (Cont u w a)
         -- ^ The 'IVar' is what we are blocked on; 'Cont' is the
         -- continuation.  This might be wrapped further if we're
         -- nested inside multiple '>>=', before finally being added
         -- to the 'IVar'.  Morally @b -> GenHaxl u w a@, but see
         -- 'IVar',

instance (Show a) => Show (Result u w a) where
  show :: Result u w a -> String
show (Done a
a) = String -> ShowS
forall r. PrintfType r => String -> r
printf String
"Done(%s)" ShowS -> ShowS
forall a b. (a -> b) -> a -> b
$ a -> String
forall a. Show a => a -> String
show a
a
  show (Throw SomeException
e) = String -> ShowS
forall r. PrintfType r => String -> r
printf String
"Throw(%s)" ShowS -> ShowS
forall a b. (a -> b) -> a -> b
$ SomeException -> String
forall a. Show a => a -> String
show SomeException
e
  show Blocked{} = String
"Blocked"

{- Note [Exception]

How do we want to represent Haxl exceptions (those that are thrown by
"throw" in the Haxl monad)?

1) Explicitly via a Throw constructor in the Result type
2) Using throwIO in the IO monad

If we did (2), we would have to use an exception handler in <*>,
because an exception in the right-hand argument of <*> should not
necessarily be thrown by the whole computation - an exception on the
left should get priority, and the left might currently be Blocked.

We must be careful about turning IO monad exceptions into Haxl
exceptions.  An IO monad exception will normally propagate right
out of runHaxl and terminate the whole computation, whereas a Haxl
exception can get dropped on the floor, if it is on the right of
<*> and the left side also throws, for example.  So turning an IO
monad exception into a Haxl exception is a dangerous thing to do.
In particular, we never want to do it for an asynchronous exception
(AllocationLimitExceeded, ThreadKilled, etc.), because these are
supposed to unconditionally terminate the computation.

There are three places where we take an arbitrary IO monad exception and
turn it into a Haxl exception:

 * wrapFetchInCatch.  Here we want to propagate a failure of the
   data source to the callers of the data source, but if the
   failure came from elsewhere (an asynchronous exception), then we
   should just propagate it

 * cacheResult (cache the results of IO operations): again,
   failures of the IO operation should be visible to the caller as
   a Haxl exception, but we exclude asynchronous exceptions from
   this.

 * unsafeToHaxlException: assume the caller knows what they're
   doing, and just wrap all exceptions.
-}


-- -----------------------------------------------------------------------------
-- Cont

-- | A data representation of a Haxl continuation.  This is to avoid
-- repeatedly traversing a left-biased tree in a continuation, leading
-- O(n^2) complexity for some pathalogical cases - see the "seql" benchmark
-- in tests/MonadBench.hs.
-- See "A Smart View on Datatypes", Jaskelioff/Rivas, ICFP'15
data Cont u w a
  = Cont (GenHaxl u w a)
  | forall b. Cont u w b :>>= (b -> GenHaxl u w a)
  | forall b. (b -> a) :<$> (Cont u w b)
  | Return (IVar u w a)

toHaxl :: Cont u w a -> GenHaxl u w a
toHaxl :: Cont u w a -> GenHaxl u w a
toHaxl (Cont GenHaxl u w a
haxl) = GenHaxl u w a
haxl
toHaxl (Cont u w b
m :>>= b -> GenHaxl u w a
k) = Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
forall u w b a. Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
toHaxlBind Cont u w b
m b -> GenHaxl u w a
k
toHaxl (b -> a
f :<$> Cont u w b
x) = (b -> a) -> Cont u w b -> GenHaxl u w a
forall a b u w. (a -> b) -> Cont u w a -> GenHaxl u w b
toHaxlFmap b -> a
f Cont u w b
x
toHaxl (Return IVar u w a
i) = IVar u w a -> GenHaxl u w a
forall u w a. IVar u w a -> GenHaxl u w a
getIVar IVar u w a
i

toHaxlBind :: Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
toHaxlBind :: Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
toHaxlBind (Cont u w b
m :>>= b -> GenHaxl u w b
k) b -> GenHaxl u w a
k2 = Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
forall u w b a. Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
toHaxlBind Cont u w b
m (b -> GenHaxl u w b
k (b -> GenHaxl u w b) -> (b -> GenHaxl u w a) -> b -> GenHaxl u w a
forall (m :: * -> *) a b c.
Monad m =>
(a -> m b) -> (b -> m c) -> a -> m c
>=> b -> GenHaxl u w a
k2)
toHaxlBind (Cont GenHaxl u w b
haxl) b -> GenHaxl u w a
k = GenHaxl u w b
haxl GenHaxl u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= b -> GenHaxl u w a
k
toHaxlBind (b -> b
f :<$> Cont u w b
x) b -> GenHaxl u w a
k = Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
forall u w b a. Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
toHaxlBind Cont u w b
x (b -> GenHaxl u w a
k (b -> GenHaxl u w a) -> (b -> b) -> b -> GenHaxl u w a
forall b c a. (b -> c) -> (a -> b) -> a -> c
. b -> b
f)
toHaxlBind (Return IVar u w b
i) b -> GenHaxl u w a
k = IVar u w b -> GenHaxl u w b
forall u w a. IVar u w a -> GenHaxl u w a
getIVar IVar u w b
i GenHaxl u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= b -> GenHaxl u w a
k

toHaxlFmap :: (a -> b) -> Cont u w a -> GenHaxl u w b
toHaxlFmap :: (a -> b) -> Cont u w a -> GenHaxl u w b
toHaxlFmap a -> b
f (Cont u w b
m :>>= b -> GenHaxl u w a
k) = Cont u w b -> (b -> GenHaxl u w b) -> GenHaxl u w b
forall u w b a. Cont u w b -> (b -> GenHaxl u w a) -> GenHaxl u w a
toHaxlBind Cont u w b
m (b -> GenHaxl u w a
k (b -> GenHaxl u w a) -> (a -> GenHaxl u w b) -> b -> GenHaxl u w b
forall (m :: * -> *) a b c.
Monad m =>
(a -> m b) -> (b -> m c) -> a -> m c
>=> b -> GenHaxl u w b
forall (m :: * -> *) a. Monad m => a -> m a
return (b -> GenHaxl u w b) -> (a -> b) -> a -> GenHaxl u w b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. a -> b
f)
toHaxlFmap a -> b
f (Cont GenHaxl u w a
haxl) = a -> b
f (a -> b) -> GenHaxl u w a -> GenHaxl u w b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> GenHaxl u w a
haxl
toHaxlFmap a -> b
f (b -> a
g :<$> Cont u w b
x) = (b -> b) -> Cont u w b -> GenHaxl u w b
forall a b u w. (a -> b) -> Cont u w a -> GenHaxl u w b
toHaxlFmap (a -> b
f (a -> b) -> (b -> a) -> b -> b
forall b c a. (b -> c) -> (a -> b) -> a -> c
. b -> a
g) Cont u w b
x
toHaxlFmap a -> b
f (Return IVar u w a
i) = a -> b
f (a -> b) -> GenHaxl u w a -> GenHaxl u w b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> IVar u w a -> GenHaxl u w a
forall u w a. IVar u w a -> GenHaxl u w a
getIVar IVar u w a
i

-- -----------------------------------------------------------------------------
-- Monad/Applicative instances

instance Monad (GenHaxl u w) where
  return :: a -> GenHaxl u w a
return a
a = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
_env -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
  GenHaxl Env u w -> IO (Result u w a)
m >>= :: GenHaxl u w a -> (a -> GenHaxl u w b) -> GenHaxl u w b
>>= a -> GenHaxl u w b
k = (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w b)) -> GenHaxl u w b)
-> (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
    Result u w a
e <- Env u w -> IO (Result u w a)
m Env u w
env
    case Result u w a
e of
      Done a
a -> GenHaxl u w b -> Env u w -> IO (Result u w b)
forall u w a. GenHaxl u w a -> Env u w -> IO (Result u w a)
unHaxl (a -> GenHaxl u w b
k a
a) Env u w
env
      Throw SomeException
e -> Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w b
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
      Blocked IVar u w b
ivar Cont u w a
cont -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
">>= Blocked" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$
        Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar (Cont u w a
cont Cont u w a -> (a -> GenHaxl u w b) -> Cont u w b
forall u w a b. Cont u w b -> (b -> GenHaxl u w a) -> Cont u w a
:>>= a -> GenHaxl u w b
k))

  -- We really want the Applicative version of >>
  >> :: GenHaxl u w a -> GenHaxl u w b -> GenHaxl u w b
(>>) = GenHaxl u w a -> GenHaxl u w b -> GenHaxl u w b
forall (f :: * -> *) a b. Applicative f => f a -> f b -> f b
(*>)

#if __GLASGOW_HASKELL__ >= 808
instance CTL.MonadFail (GenHaxl u w) where
#endif
  fail :: String -> GenHaxl u w a
fail String
msg = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
_env ->
    Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (Result u w a -> IO (Result u w a))
-> Result u w a -> IO (Result u w a)
forall a b. (a -> b) -> a -> b
$ SomeException -> Result u w a
forall u w a. SomeException -> Result u w a
Throw (SomeException -> Result u w a) -> SomeException -> Result u w a
forall a b. (a -> b) -> a -> b
$ MonadFail -> SomeException
forall e. Exception e => e -> SomeException
toException (MonadFail -> SomeException) -> MonadFail -> SomeException
forall a b. (a -> b) -> a -> b
$ ProfileLabel -> MonadFail
MonadFail (ProfileLabel -> MonadFail) -> ProfileLabel -> MonadFail
forall a b. (a -> b) -> a -> b
$ String -> ProfileLabel
Text.pack String
msg

instance Functor (GenHaxl u w) where
  fmap :: (a -> b) -> GenHaxl u w a -> GenHaxl u w b
fmap a -> b
f (GenHaxl Env u w -> IO (Result u w a)
m) = (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w b)) -> GenHaxl u w b)
-> (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
    Result u w a
r <- Env u w -> IO (Result u w a)
m Env u w
env
    case Result u w a
r of
      Done a
a -> Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (b -> Result u w b
forall u w a. a -> Result u w a
Done (a -> b
f a
a))
      Throw SomeException
e -> Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w b
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
      Blocked IVar u w b
ivar Cont u w a
cont -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"fmap Blocked" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$
        Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar (a -> b
f (a -> b) -> Cont u w a -> Cont u w b
forall u w a b. (b -> a) -> Cont u w b -> Cont u w a
:<$> Cont u w a
cont))

instance Applicative (GenHaxl u w) where
  pure :: a -> GenHaxl u w a
pure = a -> GenHaxl u w a
forall (m :: * -> *) a. Monad m => a -> m a
return
  GenHaxl Env u w -> IO (Result u w (a -> b))
ff <*> :: GenHaxl u w (a -> b) -> GenHaxl u w a -> GenHaxl u w b
<*> GenHaxl Env u w -> IO (Result u w a)
aa = (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w b)) -> GenHaxl u w b)
-> (Env u w -> IO (Result u w b)) -> GenHaxl u w b
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
    Result u w (a -> b)
rf <- Env u w -> IO (Result u w (a -> b))
ff Env u w
env
    case Result u w (a -> b)
rf of
      Done a -> b
f -> do
        Result u w a
ra <- Env u w -> IO (Result u w a)
aa Env u w
env
        case Result u w a
ra of
          Done a
a -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Done/Done" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$ Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (b -> Result u w b
forall u w a. a -> Result u w a
Done (a -> b
f a
a))
          Throw SomeException
e -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Done/Throw" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$ Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w b
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
          Blocked IVar u w b
ivar Cont u w a
fcont -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Done/Blocked" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$
            Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar (a -> b
f (a -> b) -> Cont u w a -> Cont u w b
forall u w a b. (b -> a) -> Cont u w b -> Cont u w a
:<$> Cont u w a
fcont))
      Throw SomeException
e -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Throw" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$ Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w b
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
      Blocked IVar u w b
ivar1 Cont u w (a -> b)
fcont -> do
        Result u w a
ra <- Env u w -> IO (Result u w a)
aa Env u w
env
        case Result u w a
ra of
          Done a
a -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Blocked/Done" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$
            Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar1 (((a -> b) -> a -> b
forall a b. (a -> b) -> a -> b
$ a
a) ((a -> b) -> b) -> Cont u w (a -> b) -> Cont u w b
forall u w a b. (b -> a) -> Cont u w b -> Cont u w a
:<$> Cont u w (a -> b)
fcont))
          Throw SomeException
e -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Blocked/Throw" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$
            Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar1 (Cont u w (a -> b)
fcont Cont u w (a -> b) -> ((a -> b) -> GenHaxl u w b) -> Cont u w b
forall u w a b. Cont u w b -> (b -> GenHaxl u w a) -> Cont u w a
:>>= (\a -> b
_ -> SomeException -> GenHaxl u w b
forall e u w a. Exception e => e -> GenHaxl u w a
throw SomeException
e)))
          Blocked IVar u w b
ivar2 Cont u w a
acont -> String -> IO (Result u w b) -> IO (Result u w b)
forall a. String -> a -> a
trace_ String
"Blocked/Blocked" (IO (Result u w b) -> IO (Result u w b))
-> IO (Result u w b) -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$
            Env u w
-> IVar u w b
-> Cont u w (a -> b)
-> IVar u w b
-> Cont u w a
-> IO (Result u w b)
forall u w c a b d.
Env u w
-> IVar u w c
-> Cont u w (a -> b)
-> IVar u w d
-> Cont u w a
-> IO (Result u w b)
blockedBlocked Env u w
env IVar u w b
ivar1 Cont u w (a -> b)
fcont IVar u w b
ivar2 Cont u w a
acont
             -- Note [Blocked/Blocked]

instance Semigroup a => Semigroup (GenHaxl u w a) where
  <> :: GenHaxl u w a -> GenHaxl u w a -> GenHaxl u w a
(<>) = (a -> a -> a) -> GenHaxl u w a -> GenHaxl u w a -> GenHaxl u w a
forall (f :: * -> *) a b c.
Applicative f =>
(a -> b -> c) -> f a -> f b -> f c
liftA2 a -> a -> a
forall a. Semigroup a => a -> a -> a
(<>)

instance Monoid a => Monoid (GenHaxl u w a) where
  mempty :: GenHaxl u w a
mempty = a -> GenHaxl u w a
forall (f :: * -> *) a. Applicative f => a -> f a
pure a
forall a. Monoid a => a
mempty
  mappend :: GenHaxl u w a -> GenHaxl u w a -> GenHaxl u w a
mappend = (a -> a -> a) -> GenHaxl u w a -> GenHaxl u w a -> GenHaxl u w a
forall (f :: * -> *) a b c.
Applicative f =>
(a -> b -> c) -> f a -> f b -> f c
liftA2 a -> a -> a
forall a. Monoid a => a -> a -> a
mappend

blockedBlocked
  :: Env u w
  -> IVar u w c
  -> Cont u w (a -> b)
  -> IVar u w d
  -> Cont u w a
  -> IO (Result u w b)
blockedBlocked :: Env u w
-> IVar u w c
-> Cont u w (a -> b)
-> IVar u w d
-> Cont u w a
-> IO (Result u w b)
blockedBlocked Env u w
_ IVar u w c
_ (Return IVar u w (a -> b)
i) IVar u w d
ivar2 Cont u w a
acont =
  Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w d -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w d
ivar2 (Cont u w a
acont Cont u w a -> (a -> GenHaxl u w b) -> Cont u w b
forall u w a b. Cont u w b -> (b -> GenHaxl u w a) -> Cont u w a
:>>= IVar u w (a -> b) -> a -> GenHaxl u w b
forall u w a b. IVar u w (a -> b) -> a -> GenHaxl u w b
getIVarApply IVar u w (a -> b)
i))
blockedBlocked Env u w
_ IVar u w c
_ (b -> a -> b
g :<$> Return IVar u w b
i) IVar u w d
ivar2 Cont u w a
acont =
  Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w d -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w d
ivar2 (Cont u w a
acont Cont u w a -> (a -> GenHaxl u w b) -> Cont u w b
forall u w a b. Cont u w b -> (b -> GenHaxl u w a) -> Cont u w a
:>>= \ a
a -> (\b
f -> b -> a -> b
g b
f a
a) (b -> b) -> GenHaxl u w b -> GenHaxl u w b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> IVar u w b -> GenHaxl u w b
forall u w a. IVar u w a -> GenHaxl u w a
getIVar IVar u w b
i))
blockedBlocked Env u w
env IVar u w c
ivar1 Cont u w (a -> b)
fcont IVar u w d
ivar2 Cont u w a
acont = do
  IVar u w (a -> b)
i <- IO (IVar u w (a -> b))
forall u w a. IO (IVar u w a)
newIVar
  Env u w
-> GenHaxl u w (a -> b) -> IVar u w (a -> b) -> IVar u w c -> IO ()
forall u w b a.
Env u w -> GenHaxl u w b -> IVar u w b -> IVar u w a -> IO ()
addJob Env u w
env (Cont u w (a -> b) -> GenHaxl u w (a -> b)
forall u w a. Cont u w a -> GenHaxl u w a
toHaxl Cont u w (a -> b)
fcont) IVar u w (a -> b)
i IVar u w c
ivar1
  let cont :: Cont u w b
cont = Cont u w a
acont Cont u w a -> (a -> GenHaxl u w b) -> Cont u w b
forall u w a b. Cont u w b -> (b -> GenHaxl u w a) -> Cont u w a
:>>= \a
a -> IVar u w (a -> b) -> a -> GenHaxl u w b
forall u w a b. IVar u w (a -> b) -> a -> GenHaxl u w b
getIVarApply IVar u w (a -> b)
i a
a
  Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w d -> Cont u w b -> Result u w b
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w d
ivar2 Cont u w b
cont)

-- Note [Blocked/Blocked]
--
-- This is the tricky case: we're blocked on both sides of the <*>.
-- We need to divide the computation into two pieces that may continue
-- independently when the resources they are blocked on become
-- available.  Moreover, the computation as a whole depends on the two
-- pieces.  It works like this:
--
--   ff <*> aa
--
-- becomes
--
--   (ff >>= putIVar i) <*> (a <- aa; f <- getIVar i; return (f a)
--
-- where the IVar i is a new synchronisation point.  If the right side
-- gets to the `getIVar` first, it will block until the left side has
-- called 'putIVar'.
--
-- We can also do it the other way around:
--
--   (do ff <- f; getIVar i; return (ff a)) <*> (a >>= putIVar i)
--
-- The first was slightly faster according to tests/MonadBench.hs.



-- -----------------------------------------------------------------------------
-- Env utils

-- | Extracts data from the 'Env'.
env :: (Env u w -> a) -> GenHaxl u w a
env :: (Env u w -> a) -> GenHaxl u w a
env Env u w -> a
f = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done (Env u w -> a
f Env u w
env))

-- | Returns a version of the Haxl computation which always uses the
-- provided 'Env', ignoring the one specified by 'runHaxl'.
withEnv :: Env u w -> GenHaxl u w a -> GenHaxl u w a
withEnv :: Env u w -> GenHaxl u w a -> GenHaxl u w a
withEnv Env u w
newEnv (GenHaxl Env u w -> IO (Result u w a)
m) = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
_env -> do
  Result u w a
r <- Env u w -> IO (Result u w a)
m Env u w
newEnv
  case Result u w a
r of
    Done a
a -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
    Throw SomeException
e -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w a
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
    Blocked IVar u w b
ivar Cont u w a
k ->
      Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w a -> Result u w a
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar (GenHaxl u w a -> Cont u w a
forall u w a. GenHaxl u w a -> Cont u w a
Cont (Env u w -> GenHaxl u w a -> GenHaxl u w a
forall u w a. Env u w -> GenHaxl u w a -> GenHaxl u w a
withEnv Env u w
newEnv (Cont u w a -> GenHaxl u w a
forall u w a. Cont u w a -> GenHaxl u w a
toHaxl Cont u w a
k))))

nextCallId :: Env u w -> IO CallId
nextCallId :: Env u w -> IO CallId
nextCallId Env u w
env = IORef CallId -> (CallId -> (CallId, CallId)) -> IO CallId
forall a b. IORef a -> (a -> (a, b)) -> IO b
atomicModifyIORef' (Env u w -> IORef CallId
forall u w. Env u w -> IORef CallId
callIdRef Env u w
env) ((CallId -> (CallId, CallId)) -> IO CallId)
-> (CallId -> (CallId, CallId)) -> IO CallId
forall a b. (a -> b) -> a -> b
$ \CallId
x -> (CallId
xCallId -> CallId -> CallId
forall a. Num a => a -> a -> a
+CallId
1,CallId
xCallId -> CallId -> CallId
forall a. Num a => a -> a -> a
+CallId
1)

-- | Runs the Haxl computation, transforming the writes within using the
-- provided function.
--
-- Memoization behavior is unchanged, meaning if a memoized computation is run
-- once inside @mapWrites@ and then once without, the writes from the second run
-- will NOT be transformed.
mapWrites :: (w -> w) -> GenHaxl u w a -> GenHaxl u w a
mapWrites :: (w -> w) -> GenHaxl u w a -> GenHaxl u w a
mapWrites w -> w
f GenHaxl u w a
action = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
curEnv -> do
  IORef (WriteTree w)
wlogs <- WriteTree w -> IO (IORef (WriteTree w))
forall a. a -> IO (IORef a)
newIORef WriteTree w
forall w. WriteTree w
NilWrites
  IORef (WriteTree w)
wlogsNoMemo <- WriteTree w -> IO (IORef (WriteTree w))
forall a. a -> IO (IORef a)
newIORef WriteTree w
forall w. WriteTree w
NilWrites
  let
    !newEnv :: Env u w
newEnv = Env u w
curEnv { writeLogsRef :: IORef (WriteTree w)
writeLogsRef = IORef (WriteTree w)
wlogs, writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRefNoMemo = IORef (WriteTree w)
wlogsNoMemo }
  GenHaxl u w a -> Env u w -> IO (Result u w a)
forall u w a. GenHaxl u w a -> Env u w -> IO (Result u w a)
unHaxl (Env u w -> Env u w -> GenHaxl u w a -> GenHaxl u w a
mapWritesImpl Env u w
curEnv Env u w
newEnv GenHaxl u w a
action) Env u w
newEnv
  where
    mapWritesImpl :: Env u w -> Env u w -> GenHaxl u w a -> GenHaxl u w a
mapWritesImpl Env u w
oldEnv Env u w
curEnv (GenHaxl Env u w -> IO (Result u w a)
m) = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
_ -> do
      let
        pushTransformedWrites :: IO ()
pushTransformedWrites = do
          WriteTree w
wt <- IORef (WriteTree w) -> IO (WriteTree w)
forall a. IORef a -> IO a
readIORef (IORef (WriteTree w) -> IO (WriteTree w))
-> IORef (WriteTree w) -> IO (WriteTree w)
forall a b. (a -> b) -> a -> b
$ Env u w -> IORef (WriteTree w)
forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef Env u w
curEnv
          WriteTree w -> IORef (WriteTree w) -> IO ()
forall w. WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef ((w -> w) -> WriteTree w -> WriteTree w
forall w. (w -> w) -> WriteTree w -> WriteTree w
mapWriteTree w -> w
f WriteTree w
wt) (Env u w -> IORef (WriteTree w)
forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef Env u w
oldEnv)
          WriteTree w
wtNoMemo <- IORef (WriteTree w) -> IO (WriteTree w)
forall a. IORef a -> IO a
readIORef (IORef (WriteTree w) -> IO (WriteTree w))
-> IORef (WriteTree w) -> IO (WriteTree w)
forall a b. (a -> b) -> a -> b
$ Env u w -> IORef (WriteTree w)
forall u w. Env u w -> IORef (WriteTree w)
writeLogsRefNoMemo Env u w
curEnv
          WriteTree w -> IORef (WriteTree w) -> IO ()
forall w. WriteTree w -> IORef (WriteTree w) -> IO ()
mbModifyWLRef ((w -> w) -> WriteTree w -> WriteTree w
forall w. (w -> w) -> WriteTree w -> WriteTree w
mapWriteTree w -> w
f WriteTree w
wtNoMemo) (Env u w -> IORef (WriteTree w)
forall u w. Env u w -> IORef (WriteTree w)
writeLogsRefNoMemo Env u w
oldEnv)

      Result u w a
r <- Env u w -> IO (Result u w a)
m Env u w
curEnv

      case Result u w a
r of
        Done a
a -> IO ()
pushTransformedWrites IO () -> IO (Result u w a) -> IO (Result u w a)
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
        Throw SomeException
e -> IO ()
pushTransformedWrites IO () -> IO (Result u w a) -> IO (Result u w a)
forall (m :: * -> *) a b. Monad m => m a -> m b -> m b
>> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w a
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
        Blocked IVar u w b
ivar Cont u w a
k -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return
          (IVar u w b -> Cont u w a -> Result u w a
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar (GenHaxl u w a -> Cont u w a
forall u w a. GenHaxl u w a -> Cont u w a
Cont (Env u w -> Env u w -> GenHaxl u w a -> GenHaxl u w a
mapWritesImpl Env u w
oldEnv Env u w
curEnv (Cont u w a -> GenHaxl u w a
forall u w a. Cont u w a -> GenHaxl u w a
toHaxl Cont u w a
k))))

#ifdef PROFILING
-- -----------------------------------------------------------------------------
-- CallGraph recording

-- | Returns a version of the Haxl computation which records function calls in
-- an edge list which is the function call graph. Each function that is to be
-- recorded must be wrapped with a call to @withCallGraph@.
withCallGraph
  :: Typeable a
  => (a -> Maybe Text)
  -> QualFunction
  -> GenHaxl u w a
  -> GenHaxl u w a
withCallGraph toText f a = do
  coreEnv <- env id
  -- TODO: Handle exceptions
  value <- withEnv coreEnv{currFunction = f} a
  case callGraphRef coreEnv of
    Just graph -> unsafeLiftIO $ modifyIORef' graph
      (updateCallGraph (f, currFunction coreEnv) (toText value))
    _ -> throw $ CriticalError
      "withCallGraph called without an IORef CallGraph"
  return value
  where
    updateCallGraph :: FunctionCall -> Maybe Text -> CallGraph -> CallGraph
    updateCallGraph fnCall@(childQFunc, _) (Just value) (edgeList, valueMap) =
      (fnCall : edgeList, Map.insert childQFunc value valueMap)
    updateCallGraph fnCall Nothing (edgeList, valueMap) =
      (fnCall : edgeList, valueMap)
#endif

-- -----------------------------------------------------------------------------
-- Exceptions

-- | Throw an exception in the Haxl monad
throw :: Exception e => e -> GenHaxl u w a
throw :: e -> GenHaxl u w a
throw e
e = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> Env u w -> e -> IO (Result u w a)
forall e u w a. Exception e => Env u w -> e -> IO (Result u w a)
raise Env u w
env e
e

raise :: Exception e => Env u w -> e -> IO (Result u w a)
raise :: Env u w -> e -> IO (Result u w a)
raise Env u w
env e
e = Env u w -> SomeException -> IO (Result u w a)
forall u w b. Env u w -> SomeException -> IO (Result u w b)
raiseImpl Env u w
env (e -> SomeException
forall e. Exception e => e -> SomeException
toException e
e)
#ifdef PROFILING
  currentCallStack
#endif


raiseFromIVar :: Exception e => Env u w -> IVar u w a -> e -> IO (Result u w b)
#ifdef PROFILING
raiseFromIVar env IVar{..} e =
  raiseImpl env (toException e) (ccsToStrings ivarCCS)
#else
raiseFromIVar :: Env u w -> IVar u w a -> e -> IO (Result u w b)
raiseFromIVar Env u w
env IVar u w a
_ivar e
e = Env u w -> SomeException -> IO (Result u w b)
forall u w b. Env u w -> SomeException -> IO (Result u w b)
raiseImpl Env u w
env (e -> SomeException
forall e. Exception e => e -> SomeException
toException e
e)
#endif

{-# INLINE raiseImpl #-}
#ifdef PROFILING
raiseImpl :: Env u w -> SomeException -> IO [String] -> IO (Result u w b)
raiseImpl Env{..} e getCostCentreStack
#else
raiseImpl :: Env u w -> SomeException -> IO (Result u w b)
raiseImpl :: Env u w -> SomeException -> IO (Result u w b)
raiseImpl Env{u
CallId
Maybe (DataCacheLookup w)
TVar [CompleteReq u w]
IORef CallId
IORef Profile
IORef Stats
IORef ReqCountMap
IORef (RequestStore u)
IORef (JobList u w)
IORef (WriteTree w)
HaxlDataCache u w
Flags
StateStore
ProfileCurrent
dataCacheFetchFallback :: Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: IORef (WriteTree w)
writeLogsRef :: IORef (WriteTree w)
completions :: TVar [CompleteReq u w]
submittedReqsRef :: IORef ReqCountMap
runQueueRef :: IORef (JobList u w)
reqStoreRef :: IORef (RequestStore u)
states :: StateStore
profRef :: IORef Profile
profCurrent :: ProfileCurrent
callIdRef :: IORef CallId
statsBatchIdRef :: IORef CallId
statsRef :: IORef Stats
userEnv :: u
flags :: Flags
memoKey :: CallId
memoCache :: HaxlDataCache u w
dataCache :: HaxlDataCache u w
dataCacheFetchFallback :: forall u w. Env u w -> Maybe (DataCacheLookup w)
writeLogsRefNoMemo :: forall u w. Env u w -> IORef (WriteTree w)
writeLogsRef :: forall u w. Env u w -> IORef (WriteTree w)
completions :: forall u w. Env u w -> TVar [CompleteReq u w]
submittedReqsRef :: forall u w. Env u w -> IORef ReqCountMap
runQueueRef :: forall u w. Env u w -> IORef (JobList u w)
reqStoreRef :: forall u w. Env u w -> IORef (RequestStore u)
states :: forall u w. Env u w -> StateStore
profRef :: forall u w. Env u w -> IORef Profile
profCurrent :: forall u w. Env u w -> ProfileCurrent
callIdRef :: forall u w. Env u w -> IORef CallId
statsBatchIdRef :: forall u w. Env u w -> IORef CallId
statsRef :: forall u w. Env u w -> IORef Stats
userEnv :: forall u w. Env u w -> u
flags :: forall u w. Env u w -> Flags
memoKey :: forall u w. Env u w -> CallId
memoCache :: forall u w. Env u w -> HaxlDataCache u w
dataCache :: forall u w. Env u w -> HaxlDataCache u w
..} SomeException
e
#endif
  | ReportFlag -> ReportFlags -> Bool
testReportFlag ReportFlag
ReportExceptionLabelStack (ReportFlags -> Bool) -> ReportFlags -> Bool
forall a b. (a -> b) -> a -> b
$ Flags -> ReportFlags
report Flags
flags
  , Just (HaxlException Maybe [ProfileLabel]
Nothing e
h) <- SomeException -> Maybe HaxlException
forall e. Exception e => SomeException -> Maybe e
fromException SomeException
e = do
    let stk :: [ProfileLabel]
stk = NonEmpty ProfileLabel -> [ProfileLabel]
forall a. NonEmpty a -> [a]
NonEmpty.toList (NonEmpty ProfileLabel -> [ProfileLabel])
-> NonEmpty ProfileLabel -> [ProfileLabel]
forall a b. (a -> b) -> a -> b
$ ProfileCurrent -> NonEmpty ProfileLabel
profLabelStack ProfileCurrent
profCurrent
    Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (Result u w b -> IO (Result u w b))
-> Result u w b -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$ SomeException -> Result u w b
forall u w a. SomeException -> Result u w a
Throw (SomeException -> Result u w b) -> SomeException -> Result u w b
forall a b. (a -> b) -> a -> b
$ HaxlException -> SomeException
forall e. Exception e => e -> SomeException
toException (HaxlException -> SomeException) -> HaxlException -> SomeException
forall a b. (a -> b) -> a -> b
$ Maybe [ProfileLabel] -> e -> HaxlException
forall e.
MiddleException e =>
Maybe [ProfileLabel] -> e -> HaxlException
HaxlException ([ProfileLabel] -> Maybe [ProfileLabel]
forall a. a -> Maybe a
Just [ProfileLabel]
stk) e
h
#ifdef PROFILING
  | Just (HaxlException Nothing h) <- fromException e = do
    stk <- reverse . map Text.pack <$> getCostCentreStack
    return $ Throw $ toException $ HaxlException (Just stk) h
#endif
  | Bool
otherwise = Result u w b -> IO (Result u w b)
forall (m :: * -> *) a. Monad m => a -> m a
return (Result u w b -> IO (Result u w b))
-> Result u w b -> IO (Result u w b)
forall a b. (a -> b) -> a -> b
$ SomeException -> Result u w b
forall u w a. SomeException -> Result u w a
Throw SomeException
e

-- | Catch an exception in the Haxl monad
catch :: Exception e => GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
catch :: GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
catch (GenHaxl Env u w -> IO (Result u w a)
m) e -> GenHaxl u w a
h = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
   Result u w a
r <- Env u w -> IO (Result u w a)
m Env u w
env
   case Result u w a
r of
     Done a
a    -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (a -> Result u w a
forall u w a. a -> Result u w a
Done a
a)
     Throw SomeException
e | Just e
e' <- SomeException -> Maybe e
forall e. Exception e => SomeException -> Maybe e
fromException SomeException
e -> GenHaxl u w a -> Env u w -> IO (Result u w a)
forall u w a. GenHaxl u w a -> Env u w -> IO (Result u w a)
unHaxl (e -> GenHaxl u w a
h e
e') Env u w
env
             | Bool
otherwise -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w a
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
     Blocked IVar u w b
ivar Cont u w a
k -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w a -> Result u w a
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
ivar (GenHaxl u w a -> Cont u w a
forall u w a. GenHaxl u w a -> Cont u w a
Cont (GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
forall e u w a.
Exception e =>
GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
catch (Cont u w a -> GenHaxl u w a
forall u w a. Cont u w a -> GenHaxl u w a
toHaxl Cont u w a
k) e -> GenHaxl u w a
h)))

-- | Catch exceptions that satisfy a predicate
catchIf
  :: Exception e => (e -> Bool) -> GenHaxl u w a -> (e -> GenHaxl u w a)
  -> GenHaxl u w a
catchIf :: (e -> Bool)
-> GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
catchIf e -> Bool
cond GenHaxl u w a
haxl e -> GenHaxl u w a
handler =
  GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
forall e u w a.
Exception e =>
GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
catch GenHaxl u w a
haxl ((e -> GenHaxl u w a) -> GenHaxl u w a)
-> (e -> GenHaxl u w a) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \e
e -> if e -> Bool
cond e
e then e -> GenHaxl u w a
handler e
e else e -> GenHaxl u w a
forall e u w a. Exception e => e -> GenHaxl u w a
throw e
e

-- | Returns @'Left' e@ if the computation throws an exception @e@, or
-- @'Right' a@ if it returns a result @a@.
try :: Exception e => GenHaxl u w a -> GenHaxl u w (Either e a)
try :: GenHaxl u w a -> GenHaxl u w (Either e a)
try GenHaxl u w a
haxl = (a -> Either e a
forall a b. b -> Either a b
Right (a -> Either e a) -> GenHaxl u w a -> GenHaxl u w (Either e a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> GenHaxl u w a
haxl) GenHaxl u w (Either e a)
-> (e -> GenHaxl u w (Either e a)) -> GenHaxl u w (Either e a)
forall e u w a.
Exception e =>
GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
`catch` (Either e a -> GenHaxl u w (Either e a)
forall (m :: * -> *) a. Monad m => a -> m a
return (Either e a -> GenHaxl u w (Either e a))
-> (e -> Either e a) -> e -> GenHaxl u w (Either e a)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. e -> Either e a
forall a b. a -> Either a b
Left)

-- | @since 0.3.1.0
instance Catch.MonadThrow (GenHaxl u w) where throwM :: e -> GenHaxl u w a
throwM = e -> GenHaxl u w a
forall e u w a. Exception e => e -> GenHaxl u w a
Haxl.Core.Monad.throw
-- | @since 0.3.1.0
instance Catch.MonadCatch (GenHaxl u w) where catch :: GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
catch = GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
forall e u w a.
Exception e =>
GenHaxl u w a -> (e -> GenHaxl u w a) -> GenHaxl u w a
Haxl.Core.Monad.catch


-- -----------------------------------------------------------------------------
-- Unsafe operations

-- | Under ordinary circumstances this is unnecessary; users of the Haxl
-- monad should generally /not/ perform arbitrary IO.
unsafeLiftIO :: IO a -> GenHaxl u w a
unsafeLiftIO :: IO a -> GenHaxl u w a
unsafeLiftIO IO a
m = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
_env -> a -> Result u w a
forall u w a. a -> Result u w a
Done (a -> Result u w a) -> IO a -> IO (Result u w a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> IO a
m

-- | Convert exceptions in the underlying IO monad to exceptions in
-- the Haxl monad.  This is morally unsafe, because you could then
-- catch those exceptions in Haxl and observe the underlying execution
-- order.  Not to be exposed to user code.
--
-- Note: this function does not catch async exceptions. This is a flaw in Haxl
-- where it can sometimes leave the environment in a bad state when async
-- exceptions are thrown (for example the cache may think a fetch is happening
-- but the exception has stopped it). TODO would be to make Haxl async exception
-- safe and then remove the rethrowAsyncExceptions below, but for now this is
-- safer to avoid bugs. Additionally this would not protect you from async
-- exceptions thrown while executing code in the scheduler, and so relying on
-- this function to catch all async exceptions would be ambitious at best.
unsafeToHaxlException :: GenHaxl u w a -> GenHaxl u w a
unsafeToHaxlException :: GenHaxl u w a -> GenHaxl u w a
unsafeToHaxlException (GenHaxl Env u w -> IO (Result u w a)
m) = (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall u w a. (Env u w -> IO (Result u w a)) -> GenHaxl u w a
GenHaxl ((Env u w -> IO (Result u w a)) -> GenHaxl u w a)
-> (Env u w -> IO (Result u w a)) -> GenHaxl u w a
forall a b. (a -> b) -> a -> b
$ \Env u w
env -> do
  Result u w a
r <- Env u w -> IO (Result u w a)
m Env u w
env IO (Result u w a)
-> (SomeException -> IO (Result u w a)) -> IO (Result u w a)
forall e a. Exception e => IO a -> (e -> IO a) -> IO a
`Exception.catch` \SomeException
e -> do
    SomeException -> IO ()
rethrowAsyncExceptions SomeException
e
    Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (SomeException -> Result u w a
forall u w a. SomeException -> Result u w a
Throw SomeException
e)
  case Result u w a
r of
    Blocked IVar u w b
cvar Cont u w a
c ->
      Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return (IVar u w b -> Cont u w a -> Result u w a
forall u w a b. IVar u w b -> Cont u w a -> Result u w a
Blocked IVar u w b
cvar (GenHaxl u w a -> Cont u w a
forall u w a. GenHaxl u w a -> Cont u w a
Cont (GenHaxl u w a -> GenHaxl u w a
forall u w a. GenHaxl u w a -> GenHaxl u w a
unsafeToHaxlException (Cont u w a -> GenHaxl u w a
forall u w a. Cont u w a -> GenHaxl u w a
toHaxl Cont u w a
c))))
    Result u w a
other -> Result u w a -> IO (Result u w a)
forall (m :: * -> *) a. Monad m => a -> m a
return Result u w a
other

-- | Like 'try', but lifts all exceptions into the 'HaxlException'
-- hierarchy.  Uses 'unsafeToHaxlException' internally.  Typically
-- this is used at the top level of a Haxl computation, to ensure that
-- all exceptions are caught.
tryToHaxlException :: GenHaxl u w a -> GenHaxl u w (Either HaxlException a)
tryToHaxlException :: GenHaxl u w a -> GenHaxl u w (Either HaxlException a)
tryToHaxlException GenHaxl u w a
h = (SomeException -> HaxlException)
-> Either SomeException a -> Either HaxlException a
forall (a :: * -> * -> *) b c d.
ArrowChoice a =>
a b c -> a (Either b d) (Either c d)
left SomeException -> HaxlException
asHaxlException (Either SomeException a -> Either HaxlException a)
-> GenHaxl u w (Either SomeException a)
-> GenHaxl u w (Either HaxlException a)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> GenHaxl u w a -> GenHaxl u w (Either SomeException a)
forall e u w a.
Exception e =>
GenHaxl u w a -> GenHaxl u w (Either e a)
try (GenHaxl u w a -> GenHaxl u w a
forall u w a. GenHaxl u w a -> GenHaxl u w a
unsafeToHaxlException GenHaxl u w a
h)


-- -----------------------------------------------------------------------------

-- | Dump the contents of the cache as Haskell code that, when
-- compiled and run, will recreate the same cache contents.  For
-- example, the generated code looks something like this:
--
-- > loadCache :: GenHaxl u w ()
-- > loadCache = do
-- >   cacheRequest (ListWombats 3) (Right ([1,2,3]))
-- >   cacheRequest (CountAardvarks "abcabc") (Right (2))
--
dumpCacheAsHaskell :: GenHaxl u w String
dumpCacheAsHaskell :: GenHaxl u w String
dumpCacheAsHaskell =
    String -> String -> String -> GenHaxl u w String
forall u w. String -> String -> String -> GenHaxl u w String
dumpCacheAsHaskellFn String
"loadCache" String
"GenHaxl u w ()" String
"cacheRequest"

-- | Dump the contents of the cache as Haskell code that, when
-- compiled and run, will recreate the same cache contents.
-- Does not take into account the writes done as part of the computation.
--
-- Takes the name and type for the resulting function as arguments.
-- Also takes the cacheFn to use, we can use either @cacheRequest@ or
-- @dupableCacheRequest@.
dumpCacheAsHaskellFn :: String -> String -> String -> GenHaxl u w String
dumpCacheAsHaskellFn :: String -> String -> String -> GenHaxl u w String
dumpCacheAsHaskellFn String
fnName String
fnType String
cacheFn = do
  HaxlDataCache u w
cache <- (Env u w -> HaxlDataCache u w) -> GenHaxl u w (HaxlDataCache u w)
forall u w a. (Env u w -> a) -> GenHaxl u w a
env Env u w -> HaxlDataCache u w
forall u w. Env u w -> HaxlDataCache u w
dataCache  -- NB. dataCache, not memoCache.  We ignore memoized
                       -- results when dumping the cache.
  let
    readIVar :: DataCacheItem u w b -> IO (Maybe (Either SomeException b))
readIVar (DataCacheItem IVar{ivarRef :: forall u w a. IVar u w a -> IORef (IVarContents u w a)
ivarRef = !IORef (IVarContents u w b)
ref} CallId
_) = do
      IVarContents u w b
r <- IORef (IVarContents u w b) -> IO (IVarContents u w b)
forall a. IORef a -> IO a
readIORef IORef (IVarContents u w b)
ref
      case IVarContents u w b
r of
        IVarFull (Ok b
a WriteTree w
_) -> Maybe (Either SomeException b)
-> IO (Maybe (Either SomeException b))
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException b -> Maybe (Either SomeException b)
forall a. a -> Maybe a
Just (b -> Either SomeException b
forall a b. b -> Either a b
Right b
a))
        IVarFull (ThrowHaxl SomeException
e WriteTree w
_) -> Maybe (Either SomeException b)
-> IO (Maybe (Either SomeException b))
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException b -> Maybe (Either SomeException b)
forall a. a -> Maybe a
Just (SomeException -> Either SomeException b
forall a b. a -> Either a b
Left SomeException
e))
        IVarFull (ThrowIO SomeException
e) -> Maybe (Either SomeException b)
-> IO (Maybe (Either SomeException b))
forall (m :: * -> *) a. Monad m => a -> m a
return (Either SomeException b -> Maybe (Either SomeException b)
forall a. a -> Maybe a
Just (SomeException -> Either SomeException b
forall a b. a -> Either a b
Left SomeException
e))
        IVarEmpty JobList u w
_ -> Maybe (Either SomeException b)
-> IO (Maybe (Either SomeException b))
forall (m :: * -> *) a. Monad m => a -> m a
return Maybe (Either SomeException b)
forall a. Maybe a
Nothing

    mk_cr :: (String, Either SomeException String) -> Doc
mk_cr (String
req, Either SomeException String
res) =
      String -> Doc
text String
cacheFn Doc -> Doc -> Doc
<+> Doc -> Doc
parens (String -> Doc
text String
req) Doc -> Doc -> Doc
<+> Doc -> Doc
parens (Either SomeException String -> Doc
forall a. Show a => Either a String -> Doc
result Either SomeException String
res)
    result :: Either a String -> Doc
result (Left a
e) = String -> Doc
text String
"except" Doc -> Doc -> Doc
<+> Doc -> Doc
parens (String -> Doc
text (a -> String
forall a. Show a => a -> String
show a
e))
    result (Right String
s) = String -> Doc
text String
"Right" Doc -> Doc -> Doc
<+> Doc -> Doc
parens (String -> Doc
text String
s)

  [(TypeRep, [(String, Either SomeException String)])]
entries <- IO [(TypeRep, [(String, Either SomeException String)])]
-> GenHaxl u w [(TypeRep, [(String, Either SomeException String)])]
forall a u w. IO a -> GenHaxl u w a
unsafeLiftIO (IO [(TypeRep, [(String, Either SomeException String)])]
 -> GenHaxl
      u w [(TypeRep, [(String, Either SomeException String)])])
-> IO [(TypeRep, [(String, Either SomeException String)])]
-> GenHaxl u w [(TypeRep, [(String, Either SomeException String)])]
forall a b. (a -> b) -> a -> b
$ do
    HaxlDataCache u w
-> (forall a.
    DataCacheItem u w a -> IO (Maybe (Either SomeException a)))
-> IO [(TypeRep, [(String, Either SomeException String)])]
forall (res :: * -> *).
DataCache res
-> (forall a. res a -> IO (Maybe (Either SomeException a)))
-> IO [(TypeRep, [(String, Either SomeException String)])]
showCache HaxlDataCache u w
cache forall a.
DataCacheItem u w a -> IO (Maybe (Either SomeException a))
forall u w b.
DataCacheItem u w b -> IO (Maybe (Either SomeException b))
readIVar

  let
    body :: Doc
body = if [(TypeRep, [(String, Either SomeException String)])] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [(TypeRep, [(String, Either SomeException String)])]
entries
      then String -> Doc
text String
"return ()"
      else [Doc] -> Doc
vcat (((String, Either SomeException String) -> Doc)
-> [(String, Either SomeException String)] -> [Doc]
forall a b. (a -> b) -> [a] -> [b]
map (String, Either SomeException String) -> Doc
mk_cr (((TypeRep, [(String, Either SomeException String)])
 -> [(String, Either SomeException String)])
-> [(TypeRep, [(String, Either SomeException String)])]
-> [(String, Either SomeException String)]
forall (t :: * -> *) a b. Foldable t => (a -> [b]) -> t a -> [b]
concatMap (TypeRep, [(String, Either SomeException String)])
-> [(String, Either SomeException String)]
forall a b. (a, b) -> b
snd [(TypeRep, [(String, Either SomeException String)])]
entries))

  String -> GenHaxl u w String
forall (m :: * -> *) a. Monad m => a -> m a
return (String -> GenHaxl u w String) -> String -> GenHaxl u w String
forall a b. (a -> b) -> a -> b
$ Doc -> String
forall a. Show a => a -> String
show (Doc -> String) -> Doc -> String
forall a b. (a -> b) -> a -> b
$
    String -> Doc
text (String
fnName String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
" :: " String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
fnType) Doc -> Doc -> Doc
$$
    String -> Doc
text (String
fnName String -> ShowS
forall a. [a] -> [a] -> [a]
++ String
" = do") Doc -> Doc -> Doc
$$
      CallId -> Doc -> Doc
nest CallId
2 Doc
body Doc -> Doc -> Doc
$$
    String -> Doc
text String
"" -- final newline