mirror of
https://github.com/facebook/Haxl.git
synced 2025-01-08 19:00:25 +03:00
Haxl 2
Summary: This is a complete reworking of the way that Haxl schedules I/O. The main benefits are: * Data fetches are no longer organised into rounds, but can be arbitrarily overlapped with each other and with computation. The scheduler supports an arbitrary queue of work items which it can evaluate while data-fetching is taking place in the background. To take advantage of this, data sources must implement a new form of `PerformFetch`, namely `BackgroundFetch`. The old forms of `PerformFetch` are still supported, but won't benefit from any additional concurrency. * It is now possible to specify on a per-data-source basis whether fetching should be optimised for batching or for latency. A request to a data source that doesn't benefit from batching can be submitted immediately. This is done with the new `schedulerHint` method of `DataSource`. Reviewed By: niteria Differential Revision: D4938005 fbshipit-source-id: 96f12ad05ee62d62474ee4cc1215f19d0a6fcdf3
This commit is contained in:
parent
762d110041
commit
b67f7f6370
15
Haxl/Core.hs
15
Haxl/Core.hs
@ -39,20 +39,16 @@ module Haxl.Core (
|
||||
|
||||
-- ** Statistics
|
||||
Stats(..),
|
||||
RoundStats(..),
|
||||
DataSourceRoundStats(..),
|
||||
FetchStats(..),
|
||||
Microseconds,
|
||||
emptyStats,
|
||||
numRounds,
|
||||
numFetches,
|
||||
ppStats,
|
||||
ppRoundStats,
|
||||
ppDataSourceRoundStats,
|
||||
ppFetchStats,
|
||||
Profile,
|
||||
emptyProfile,
|
||||
profile,
|
||||
profileRound,
|
||||
profileCache,
|
||||
ProfileLabel,
|
||||
ProfileData(..),
|
||||
emptyProfileData,
|
||||
@ -72,17 +68,14 @@ module Haxl.Core (
|
||||
BlockedFetch(..),
|
||||
PerformFetch(..),
|
||||
StateKey(..),
|
||||
SchedulerHint(..),
|
||||
|
||||
-- ** Result variables
|
||||
ResultVar(..),
|
||||
newEmptyResult,
|
||||
newResult,
|
||||
mkResultVar,
|
||||
putFailure,
|
||||
putResult,
|
||||
putSuccess,
|
||||
takeResult,
|
||||
tryReadResult,
|
||||
tryTakeResult,
|
||||
|
||||
-- ** Default fetch implementations
|
||||
asyncFetch, asyncFetchWithDispatch, asyncFetchAcquireRelease,
|
||||
|
@ -8,6 +8,7 @@
|
||||
{-# LANGUAGE CPP #-}
|
||||
{-# LANGUAGE ExistentialQuantification #-}
|
||||
{-# LANGUAGE RankNTypes #-}
|
||||
{-# LANGUAGE ScopedTypeVariables #-}
|
||||
{-# OPTIONS_GHC -fno-warn-orphans #-}
|
||||
|
||||
-- | A cache mapping data requests to their results. This module is
|
||||
@ -24,6 +25,7 @@ module Haxl.Core.DataCache
|
||||
, showCache
|
||||
) where
|
||||
|
||||
import Control.Exception
|
||||
import Data.Hashable
|
||||
import Prelude hiding (lookup)
|
||||
import Unsafe.Coerce
|
||||
@ -33,7 +35,6 @@ import Data.Maybe
|
||||
#if __GLASGOW_HASKELL__ < 710
|
||||
import Control.Applicative ((<$>))
|
||||
#endif
|
||||
import Control.Exception
|
||||
|
||||
import Haxl.Core.Types
|
||||
|
||||
@ -109,20 +110,22 @@ lookup req (DataCache m) =
|
||||
-- 'insertNotShowable' has been used to insert any entries.
|
||||
--
|
||||
showCache
|
||||
:: DataCache ResultVar
|
||||
:: forall res
|
||||
. DataCache res
|
||||
-> (forall a . res a -> IO (Maybe (Either SomeException a)))
|
||||
-> IO [(TypeRep, [(String, Either SomeException String)])]
|
||||
|
||||
showCache (DataCache cache) = mapM goSubCache (HashMap.toList cache)
|
||||
showCache (DataCache cache) readRes = mapM goSubCache (HashMap.toList cache)
|
||||
where
|
||||
goSubCache
|
||||
:: (TypeRep,SubCache ResultVar)
|
||||
:: (TypeRep, SubCache res)
|
||||
-> IO (TypeRep,[(String, Either SomeException String)])
|
||||
goSubCache (ty, SubCache showReq showRes hmap) = do
|
||||
elems <- catMaybes <$> mapM go (HashMap.toList hmap)
|
||||
return (ty, elems)
|
||||
where
|
||||
go (req, rvar) = do
|
||||
maybe_r <- tryReadResult rvar
|
||||
maybe_r <- readRes rvar
|
||||
case maybe_r of
|
||||
Nothing -> return Nothing
|
||||
Just (Left e) -> return (Just (showReq req, Left e))
|
||||
|
@ -53,6 +53,7 @@ module Haxl.Core.Exception (
|
||||
-- ** Internal exceptions
|
||||
CriticalError(..),
|
||||
DataSourceError(..),
|
||||
NonHaxlException(..),
|
||||
|
||||
-- ** Logic exceptions
|
||||
NotFound(..),
|
||||
|
1445
Haxl/Core/Monad.hs
1445
Haxl/Core/Monad.hs
File diff suppressed because it is too large
Load Diff
@ -20,7 +20,7 @@
|
||||
-- users should not need to import it.
|
||||
module Haxl.Core.RequestStore (
|
||||
BlockedFetches(..), RequestStore,
|
||||
noRequests, addRequest, contents
|
||||
isEmpty, noRequests, addRequest, contents
|
||||
) where
|
||||
|
||||
import Haxl.Core.Types
|
||||
@ -39,6 +39,9 @@ newtype RequestStore u = RequestStore (Map TypeRep (BlockedFetches u))
|
||||
data BlockedFetches u =
|
||||
forall r. (DataSource u r) => BlockedFetches [BlockedFetch r]
|
||||
|
||||
isEmpty :: RequestStore u -> Bool
|
||||
isEmpty (RequestStore m) = Map.null m
|
||||
|
||||
-- | A new empty 'RequestStore'.
|
||||
noRequests :: RequestStore u
|
||||
noRequests = RequestStore Map.empty
|
||||
|
@ -35,21 +35,18 @@ module Haxl.Core.Types (
|
||||
|
||||
-- * Statistics
|
||||
Stats(..),
|
||||
RoundStats(..),
|
||||
DataSourceRoundStats(..),
|
||||
FetchStats(..),
|
||||
Microseconds,
|
||||
Round,
|
||||
Timestamp,
|
||||
getTimestamp,
|
||||
emptyStats,
|
||||
numRounds,
|
||||
numFetches,
|
||||
ppStats,
|
||||
ppRoundStats,
|
||||
ppDataSourceRoundStats,
|
||||
ppFetchStats,
|
||||
Profile,
|
||||
emptyProfile,
|
||||
profile,
|
||||
profileRound,
|
||||
profileCache,
|
||||
ProfileLabel,
|
||||
ProfileData(..),
|
||||
emptyProfileData,
|
||||
@ -62,6 +59,7 @@ module Haxl.Core.Types (
|
||||
Request,
|
||||
BlockedFetch(..),
|
||||
PerformFetch(..),
|
||||
SchedulerHint(..),
|
||||
|
||||
-- * DataCache
|
||||
DataCache(..),
|
||||
@ -70,14 +68,10 @@ module Haxl.Core.Types (
|
||||
|
||||
-- * Result variables
|
||||
ResultVar(..),
|
||||
newEmptyResult,
|
||||
newResult,
|
||||
mkResultVar,
|
||||
putFailure,
|
||||
putResult,
|
||||
putSuccess,
|
||||
takeResult,
|
||||
tryReadResult,
|
||||
tryTakeResult,
|
||||
|
||||
-- * Default fetch implementations
|
||||
asyncFetch, asyncFetchWithDispatch,
|
||||
@ -94,23 +88,21 @@ module Haxl.Core.Types (
|
||||
#if __GLASGOW_HASKELL__ < 710
|
||||
import Control.Applicative
|
||||
#endif
|
||||
import Control.Concurrent.MVar
|
||||
import Control.Exception
|
||||
import Control.Monad
|
||||
import Data.Aeson
|
||||
import Data.Function (on)
|
||||
import Data.Functor.Constant
|
||||
import Data.Int
|
||||
import Data.Hashable
|
||||
import Data.HashMap.Strict (HashMap, toList)
|
||||
import Data.HashMap.Strict (HashMap)
|
||||
import qualified Data.HashMap.Strict as HashMap
|
||||
import Data.HashSet (HashSet)
|
||||
import qualified Data.HashSet as HashSet
|
||||
import Data.List (intercalate, sortBy)
|
||||
import Data.Map (Map)
|
||||
import qualified Data.Map as Map
|
||||
import Data.Text (Text, unpack)
|
||||
import Data.Typeable
|
||||
import Data.List (intercalate)
|
||||
import Data.Text (Text)
|
||||
import qualified Data.Text as Text
|
||||
import Data.Time.Clock.POSIX
|
||||
import Data.Typeable.Internal
|
||||
import Text.Printf
|
||||
|
||||
import Haxl.Core.Exception
|
||||
#if __GLASGOW_HASKELL__ < 708
|
||||
@ -127,18 +119,23 @@ data Flags = Flags
|
||||
{ trace :: {-# UNPACK #-} !Int
|
||||
-- ^ Tracing level (0 = quiet, 3 = very verbose).
|
||||
, report :: {-# UNPACK #-} !Int
|
||||
-- ^ Report level (0 = quiet, 1 = # of requests, 2 = time, 3 = # of errors,
|
||||
-- 4 = profiling, 5 = log stack traces of dataFetch calls)
|
||||
-- ^ Report level:
|
||||
-- * 0 = quiet
|
||||
-- * 1 = quiet (legacy, this used to do something)
|
||||
-- * 2 = data fetch stats & errors
|
||||
-- * 3 = (same as 2, this used to enable errors)
|
||||
-- * 4 = profiling
|
||||
-- * 5 = log stack traces of dataFetch calls
|
||||
, caching :: {-# UNPACK #-} !Int
|
||||
-- ^ Non-zero if caching is enabled. If caching is disabled, then
|
||||
-- we still do batching and de-duplication within a round, but do
|
||||
-- not cache results between rounds.
|
||||
-- we still do batching and de-duplication, but do not cache
|
||||
-- results.
|
||||
}
|
||||
|
||||
defaultFlags :: Flags
|
||||
defaultFlags = Flags
|
||||
{ trace = 0
|
||||
, report = 1
|
||||
, report = 0
|
||||
, caching = 1
|
||||
}
|
||||
|
||||
@ -161,37 +158,48 @@ ifProfiling flags = when (report flags >= 4) . void
|
||||
|
||||
#undef FUNMONAD
|
||||
|
||||
-- ---------------------------------------------------------------------------
|
||||
-- Measuring time
|
||||
|
||||
type Microseconds = Int64
|
||||
type Timestamp = Microseconds -- since an epoch
|
||||
|
||||
getTimestamp :: IO Timestamp
|
||||
getTimestamp = do
|
||||
t <- getPOSIXTime -- for now, TODO better
|
||||
return (round (t * 1000000))
|
||||
|
||||
-- ---------------------------------------------------------------------------
|
||||
-- Stats
|
||||
|
||||
type Microseconds = Int
|
||||
-- | Rounds are 1-indexed
|
||||
type Round = Int
|
||||
|
||||
-- | Stats that we collect along the way.
|
||||
newtype Stats = Stats [RoundStats]
|
||||
newtype Stats = Stats [FetchStats]
|
||||
deriving (Show, ToJSON)
|
||||
|
||||
-- | Pretty-print Stats.
|
||||
ppStats :: Stats -> String
|
||||
ppStats (Stats rss) =
|
||||
intercalate "\n"
|
||||
[ "Round: " ++ show i ++ " - " ++ ppRoundStats rs
|
||||
| (i, rs) <- zip [(1::Int)..] (filter isRoundStats (reverse rss)) ]
|
||||
[ "Fetch: " ++ show i ++ " - " ++ ppFetchStats rs
|
||||
| (i, rs) <- zip [(1::Int)..] (filter isFetchStats (reverse rss)) ]
|
||||
where
|
||||
isRoundStats RoundStats{} = True
|
||||
isRoundStats _ = False
|
||||
isFetchStats FetchStats{} = True
|
||||
isFetchStats _ = False
|
||||
|
||||
-- | Maps data source name to the number of requests made in that round.
|
||||
-- The map only contains entries for sources that made requests in that
|
||||
-- round.
|
||||
data RoundStats
|
||||
-- | Timing stats for a round of data fetching
|
||||
= RoundStats
|
||||
{ roundTime :: Microseconds
|
||||
, roundAllocation :: Int
|
||||
, roundDataSources :: HashMap Text DataSourceRoundStats
|
||||
data FetchStats
|
||||
-- | Timing stats for a (batched) data fetch
|
||||
= FetchStats
|
||||
{ fetchDataSource :: Text
|
||||
, fetchBatchSize :: {-# UNPACK #-} !Int
|
||||
, fetchStart :: !Timestamp -- TODO should be something else
|
||||
, fetchDuration :: {-# UNPACK #-} !Microseconds
|
||||
, fetchSpace :: {-# UNPACK #-} !Int64
|
||||
, fetchFailures :: {-# UNPACK #-} !Int
|
||||
}
|
||||
|
||||
-- | The stack trace of a call to 'dataFetch'. These are collected
|
||||
-- only when profiling and reportLevel is 5 or greater.
|
||||
| FetchCall
|
||||
@ -201,61 +209,35 @@ data RoundStats
|
||||
deriving (Show)
|
||||
|
||||
-- | Pretty-print RoundStats.
|
||||
ppRoundStats :: RoundStats -> String
|
||||
ppRoundStats (RoundStats t a dss) =
|
||||
show t ++ "us " ++ show a ++ " bytes\n"
|
||||
++ unlines [ " " ++ unpack nm ++ ": " ++ ppDataSourceRoundStats dsrs
|
||||
| (nm, dsrs) <- sortBy (compare `on` fst) (toList dss) ]
|
||||
ppRoundStats (FetchCall r ss) = show r ++ '\n':show ss
|
||||
ppFetchStats :: FetchStats -> String
|
||||
ppFetchStats FetchStats{..} =
|
||||
printf "%s: %d fetches (%.2fms, %d bytes, %d failures)"
|
||||
(Text.unpack fetchDataSource) fetchBatchSize
|
||||
(fromIntegral fetchDuration / 1000 :: Double) fetchSpace fetchFailures
|
||||
ppFetchStats (FetchCall r ss) = show r ++ '\n':show ss
|
||||
|
||||
instance ToJSON RoundStats where
|
||||
toJSON RoundStats{..} = object
|
||||
[ "time" .= roundTime
|
||||
, "allocation" .= roundAllocation
|
||||
, "dataSources" .= roundDataSources
|
||||
instance ToJSON FetchStats where
|
||||
toJSON FetchStats{..} = object
|
||||
[ "datasource" .= fetchDataSource
|
||||
, "fetches" .= fetchBatchSize
|
||||
, "start" .= fetchStart
|
||||
, "duration" .= fetchDuration
|
||||
, "allocation" .= fetchSpace
|
||||
, "failures" .= fetchFailures
|
||||
]
|
||||
toJSON (FetchCall req strs) = object
|
||||
[ "request" .= req
|
||||
, "stack" .= strs
|
||||
]
|
||||
|
||||
-- | Detailed stats of each data source in each round.
|
||||
data DataSourceRoundStats = DataSourceRoundStats
|
||||
{ dataSourceFetches :: Int
|
||||
, dataSourceTime :: Maybe Microseconds
|
||||
, dataSourceFailures :: Maybe Int
|
||||
, dataSourceAllocation :: Maybe Int
|
||||
} deriving (Show)
|
||||
|
||||
-- | Pretty-print DataSourceRoundStats
|
||||
ppDataSourceRoundStats :: DataSourceRoundStats -> String
|
||||
ppDataSourceRoundStats (DataSourceRoundStats fetches time failures allocs) =
|
||||
maybe id (\t s -> s ++ " (" ++ show t ++ "us)") time $
|
||||
maybe id (\a s -> s ++ " (" ++ show a ++ " bytes)") allocs $
|
||||
maybe id (\f s -> s ++ " " ++ show f ++ " failures") failures $
|
||||
show fetches ++ " fetches"
|
||||
|
||||
instance ToJSON DataSourceRoundStats where
|
||||
toJSON DataSourceRoundStats{..} = object [k .= v | (k, Just v) <-
|
||||
[ ("fetches", Just dataSourceFetches)
|
||||
, ("time", dataSourceTime)
|
||||
, ("failures", dataSourceFailures)
|
||||
, ("allocation", dataSourceAllocation)
|
||||
]]
|
||||
|
||||
fetchesInRound :: RoundStats -> Int
|
||||
fetchesInRound (RoundStats _ _ hm) =
|
||||
sum $ map dataSourceFetches $ HashMap.elems hm
|
||||
fetchesInRound _ = 0
|
||||
|
||||
emptyStats :: Stats
|
||||
emptyStats = Stats []
|
||||
|
||||
numRounds :: Stats -> Int
|
||||
numRounds (Stats rs) = length [ s | s@RoundStats{} <- rs ]
|
||||
numRounds (Stats rs) = length rs -- not really
|
||||
|
||||
numFetches :: Stats -> Int
|
||||
numFetches (Stats rs) = sum (map fetchesInRound rs)
|
||||
numFetches (Stats rs) = sum [ fetchBatchSize | FetchStats{..} <- rs ]
|
||||
|
||||
|
||||
-- ---------------------------------------------------------------------------
|
||||
@ -265,42 +247,36 @@ type ProfileLabel = Text
|
||||
type AllocCount = Int64
|
||||
type MemoHitCount = Int64
|
||||
|
||||
data Profile = Profile
|
||||
{ profileRound :: {-# UNPACK #-} !Round
|
||||
-- ^ Keep track of what the current fetch round is.
|
||||
, profile :: HashMap ProfileLabel ProfileData
|
||||
newtype Profile = Profile
|
||||
{ profile :: HashMap ProfileLabel ProfileData
|
||||
-- ^ Data on individual labels.
|
||||
, profileCache :: DataCache (Constant Round)
|
||||
-- ^ Keep track of the round requests first appear in.
|
||||
}
|
||||
|
||||
emptyProfile :: Profile
|
||||
emptyProfile = Profile 1 HashMap.empty emptyDataCache
|
||||
emptyProfile = Profile HashMap.empty
|
||||
|
||||
data ProfileData = ProfileData
|
||||
{ profileAllocs :: {-# UNPACK #-} !AllocCount
|
||||
-- ^ allocations made by this label
|
||||
, profileDeps :: HashSet ProfileLabel
|
||||
-- ^ labels that this label depends on
|
||||
, profileFetches :: Map Round (HashMap Text Int)
|
||||
-- ^ map from round to {datasource name => fetch count}
|
||||
, profileFetches :: HashMap Text Int
|
||||
-- ^ map from datasource name => fetch count
|
||||
, profileMemoHits :: {-# UNPACK #-} !MemoHitCount
|
||||
-- ^ number of hits to memoized computation at this label
|
||||
}
|
||||
deriving Show
|
||||
|
||||
emptyProfileData :: ProfileData
|
||||
emptyProfileData = ProfileData 0 HashSet.empty Map.empty 0
|
||||
emptyProfileData = ProfileData 0 HashSet.empty HashMap.empty 0
|
||||
|
||||
-- ---------------------------------------------------------------------------
|
||||
-- DataCache
|
||||
|
||||
-- | The 'DataCache' maps things of type @f a@ to @'ResultVar' a@, for
|
||||
-- any @f@ and @a@ provided @f a@ is an instance of 'Typeable'. In
|
||||
-- practice @f a@ will be a request type parameterised by its result.
|
||||
-- | A @'DataCache' res@ maps things of type @req a@ to @res a@, for
|
||||
-- any @req@ and @a@ provided @req a@ is an instance of 'Typeable'. In
|
||||
-- practice @req a@ will be a request type parameterised by its result.
|
||||
--
|
||||
-- See the definition of 'ResultVar' for more details.
|
||||
|
||||
newtype DataCache res = DataCache (HashMap TypeRep (SubCache res))
|
||||
|
||||
-- | The implementation is a two-level map: the outer level maps the
|
||||
@ -308,6 +284,7 @@ newtype DataCache res = DataCache (HashMap TypeRep (SubCache res))
|
||||
-- results. So each 'SubCache' contains requests of the same type.
|
||||
-- This works well because we only have to store the dictionaries for
|
||||
-- 'Hashable' and 'Eq' once per request type.
|
||||
--
|
||||
data SubCache res =
|
||||
forall req a . (Hashable (req a), Eq (req a), Typeable (req a)) =>
|
||||
SubCache (req a -> String) (a -> String) ! (HashMap (req a) (res a))
|
||||
@ -344,11 +321,12 @@ class (DataSourceName req, StateKey req, ShowP req) => DataSource u req where
|
||||
-- ^ Tracing flags.
|
||||
-> u
|
||||
-- ^ User environment.
|
||||
-> [BlockedFetch req]
|
||||
-- ^ Requests to fetch.
|
||||
-> PerformFetch
|
||||
-> PerformFetch req
|
||||
-- ^ Fetch the data; see 'PerformFetch'.
|
||||
|
||||
schedulerHint :: u -> SchedulerHint req
|
||||
schedulerHint _ = TryToBatch
|
||||
|
||||
class DataSourceName req where
|
||||
-- | The name of this 'DataSource', used in tracing and stats. Must
|
||||
-- take a dummy request.
|
||||
@ -369,29 +347,46 @@ type Request req a =
|
||||
, Show a
|
||||
)
|
||||
|
||||
-- | A data source can fetch data in one of two ways.
|
||||
--
|
||||
-- * Synchronously ('SyncFetch'): the fetching operation is an
|
||||
-- @'IO' ()@ that fetches all the data and then returns.
|
||||
--
|
||||
-- * Asynchronously ('AsyncFetch'): we can do something else while the
|
||||
-- data is being fetched. The fetching operation takes an @'IO' ()@ as
|
||||
-- an argument, which is the operation to perform while the data is
|
||||
-- being fetched.
|
||||
--
|
||||
-- See 'syncFetch' and 'asyncFetch' for example usage.
|
||||
--
|
||||
data PerformFetch
|
||||
= SyncFetch (IO ())
|
||||
| AsyncFetch (IO () -> IO ())
|
||||
-- | Hints to the scheduler about this data source
|
||||
data SchedulerHint (req :: * -> *)
|
||||
= TryToBatch
|
||||
-- ^ Hold data-source requests while we execute as much as we can, so
|
||||
-- that we can hopefully collect more requests to batch.
|
||||
| SubmitImmediately
|
||||
-- ^ Submit a request via fetch as soon as we have one, don't try to
|
||||
-- batch multiple requests. This is really only useful if the data source
|
||||
-- returns BackgroundFetch, otherwise requests to this data source will
|
||||
-- be performed synchronously, one at a time.
|
||||
|
||||
-- | A data source can fetch data in one of four ways.
|
||||
--
|
||||
data PerformFetch req
|
||||
= SyncFetch ([BlockedFetch req] -> IO ())
|
||||
-- ^ Fully synchronous, returns only when all the data is fetched.
|
||||
-- See 'syncFetch' for an example.
|
||||
| AsyncFetch ([BlockedFetch req] -> IO () -> IO ())
|
||||
-- ^ Asynchronous; performs an arbitrary IO action while the data
|
||||
-- is being fetched, but only returns when all the data is
|
||||
-- fetched. See 'asyncFetch' for an example.
|
||||
| BackgroundFetch ([BlockedFetch req] -> IO ())
|
||||
-- ^ Fetches the data in the background, calling 'putResult' at
|
||||
-- any time in the future. This is the best kind of fetch,
|
||||
-- because it provides the most concurrency.
|
||||
| FutureFetch ([BlockedFetch req] -> IO (IO ()))
|
||||
-- ^ Returns an IO action that, when performed, waits for the data
|
||||
-- to be received. This is the second-best type of fetch, because
|
||||
-- the scheduler still has to perform the blocking wait at some
|
||||
-- point in the future, and when it has multiple blocking waits to
|
||||
-- perform, it can't know which one will return first.
|
||||
--
|
||||
-- Why not just forkIO the IO action to make a FutureFetch into a
|
||||
-- BackgroundFetch? The blocking wait will probably do a safe FFI
|
||||
-- call, which means it needs its own OS thread. If we don't want
|
||||
-- to create an arbitrary number of OS threads, then FutureFetch
|
||||
-- enables all the blocking waits to be done on a single thread.
|
||||
-- Also, you might have a data source that requires all calls to
|
||||
-- be made in the same OS thread.
|
||||
|
||||
-- Why does AsyncFetch contain a `IO () -> IO ()` rather than the
|
||||
-- alternative approach of returning the `IO` action to retrieve the
|
||||
-- results, which might seem better: `IO (IO ())`? The point is that
|
||||
-- this allows the data source to acquire resources for the purpose of
|
||||
-- this fetching round using the standard `bracket` pattern, so it can
|
||||
-- ensure that the resources acquired are properly released even if
|
||||
-- other data sources fail.
|
||||
|
||||
-- | A 'BlockedFetch' is a pair of
|
||||
--
|
||||
@ -412,40 +407,15 @@ data PerformFetch
|
||||
--
|
||||
data BlockedFetch r = forall a. BlockedFetch (r a) (ResultVar a)
|
||||
|
||||
-- | Function for easily setting a fetch to a particular exception
|
||||
setError :: (Exception e) => (forall a. r a -> e) -> BlockedFetch r -> IO ()
|
||||
setError e (BlockedFetch req m) = putFailure m (e req)
|
||||
|
||||
except :: (Exception e) => e -> Either SomeException a
|
||||
except = Left . toException
|
||||
-- -----------------------------------------------------------------------------
|
||||
-- ResultVar
|
||||
|
||||
-- | A sink for the result of a data fetch in 'BlockedFetch'
|
||||
newtype ResultVar a = ResultVar (MVar (Either SomeException a))
|
||||
newtype ResultVar a = ResultVar (Either SomeException a -> IO ())
|
||||
|
||||
-- Why do we need an 'MVar' here? The reason is that the
|
||||
-- cache serves two purposes:
|
||||
--
|
||||
-- 1. To cache the results of requests that were submitted in a previous round.
|
||||
--
|
||||
-- 2. To remember requests that have been encountered in the current round but
|
||||
-- are not yet submitted, so that if we see the request again we can make
|
||||
-- sure that we only submit it once.
|
||||
--
|
||||
-- Storing the result as an 'MVar' gives two benefits:
|
||||
--
|
||||
-- * We can tell the difference between (1) and (2) by testing whether the
|
||||
-- 'MVar' is empty. See 'Haxl.Fetch.cached'.
|
||||
--
|
||||
-- * In the case of (2), we don't have to update the cache again after the
|
||||
-- current round, and after the round we can read the result of each request
|
||||
-- from its 'MVar'. All instances of identical requests will share the same
|
||||
-- 'MVar' to obtain the result.
|
||||
|
||||
newResult :: a -> IO (ResultVar a)
|
||||
newResult x = ResultVar <$> newMVar (Right x)
|
||||
|
||||
newEmptyResult :: IO (ResultVar a)
|
||||
newEmptyResult = ResultVar <$> newEmptyMVar
|
||||
mkResultVar :: (Either SomeException a -> IO ()) -> ResultVar a
|
||||
mkResultVar = ResultVar
|
||||
|
||||
putFailure :: (Exception e) => ResultVar a -> e -> IO ()
|
||||
putFailure r = putResult r . except
|
||||
@ -454,23 +424,23 @@ putSuccess :: ResultVar a -> a -> IO ()
|
||||
putSuccess r = putResult r . Right
|
||||
|
||||
putResult :: ResultVar a -> Either SomeException a -> IO ()
|
||||
putResult (ResultVar var) = putMVar var
|
||||
putResult (ResultVar io) res = io res
|
||||
|
||||
takeResult :: ResultVar a -> IO (Either SomeException a)
|
||||
takeResult (ResultVar var) = takeMVar var
|
||||
-- | Function for easily setting a fetch to a particular exception
|
||||
setError :: (Exception e) => (forall a. r a -> e) -> BlockedFetch r -> IO ()
|
||||
setError e (BlockedFetch req m) = putFailure m (e req)
|
||||
|
||||
tryReadResult :: ResultVar a -> IO (Maybe (Either SomeException a))
|
||||
tryReadResult (ResultVar var) = tryReadMVar var
|
||||
except :: (Exception e) => e -> Either SomeException a
|
||||
except = Left . toException
|
||||
|
||||
tryTakeResult :: ResultVar a -> IO (Maybe (Either SomeException a))
|
||||
tryTakeResult (ResultVar var) = tryTakeMVar var
|
||||
|
||||
-- -----------------------------------------------------------------------------
|
||||
-- Fetch templates
|
||||
|
||||
stubFetch
|
||||
:: (Exception e) => (forall a. r a -> e)
|
||||
-> State r -> Flags -> u -> [BlockedFetch r] -> PerformFetch
|
||||
stubFetch e _state _flags _si bfs = SyncFetch $ mapM_ (setError e) bfs
|
||||
-> State r -> Flags -> u -> PerformFetch r
|
||||
stubFetch e _state _flags _si = SyncFetch $ mapM_ (setError e)
|
||||
|
||||
-- | Common implementation templates for 'fetch' of 'DataSource'.
|
||||
--
|
||||
@ -503,10 +473,7 @@ asyncFetchWithDispatch
|
||||
-> u
|
||||
-- ^ Currently unused.
|
||||
|
||||
-> [BlockedFetch request]
|
||||
-- ^ Requests to submit.
|
||||
|
||||
-> PerformFetch
|
||||
-> PerformFetch request
|
||||
|
||||
asyncFetch, syncFetch
|
||||
:: ((service -> IO ()) -> IO ())
|
||||
@ -527,29 +494,26 @@ asyncFetch, syncFetch
|
||||
-> u
|
||||
-- ^ Currently unused.
|
||||
|
||||
-> [BlockedFetch request]
|
||||
-- ^ Requests to submit.
|
||||
|
||||
-> PerformFetch
|
||||
-> PerformFetch request
|
||||
|
||||
asyncFetchWithDispatch
|
||||
withService dispatch wait enqueue _state _flags _si requests =
|
||||
AsyncFetch $ \inner -> withService $ \service -> do
|
||||
withService dispatch wait enqueue _state _flags _si =
|
||||
AsyncFetch $ \requests inner -> withService $ \service -> do
|
||||
getResults <- mapM (submitFetch service enqueue) requests
|
||||
dispatch service
|
||||
inner
|
||||
wait service
|
||||
sequence_ getResults
|
||||
|
||||
asyncFetch withService wait enqueue _state _flags _si requests =
|
||||
AsyncFetch $ \inner -> withService $ \service -> do
|
||||
asyncFetch withService wait enqueue _state _flags _si =
|
||||
AsyncFetch $ \requests inner -> withService $ \service -> do
|
||||
getResults <- mapM (submitFetch service enqueue) requests
|
||||
inner
|
||||
wait service
|
||||
sequence_ getResults
|
||||
|
||||
syncFetch withService dispatch enqueue _state _flags _si requests =
|
||||
SyncFetch . withService $ \service -> do
|
||||
syncFetch withService dispatch enqueue _state _flags _si =
|
||||
SyncFetch $ \requests -> withService $ \service -> do
|
||||
getResults <- mapM (submitFetch service enqueue) requests
|
||||
dispatch service
|
||||
sequence_ getResults
|
||||
@ -566,7 +530,7 @@ the same round.
|
||||
'asyncFetchAcquireRelease' behaves like the following:
|
||||
|
||||
> asyncFetchAcquireRelease acquire release dispatch wait enqueue =
|
||||
> AsyncFetch $ \inner ->
|
||||
> AsyncFetch $ \requests inner ->
|
||||
> bracket acquire release $ \service -> do
|
||||
> getResults <- mapM (submitFetch service enqueue) requests
|
||||
> dispatch service
|
||||
@ -603,14 +567,11 @@ asyncFetchAcquireRelease
|
||||
-> u
|
||||
-- ^ Currently unused.
|
||||
|
||||
-> [BlockedFetch request]
|
||||
-- ^ Requests to submit.
|
||||
|
||||
-> PerformFetch
|
||||
-> PerformFetch request
|
||||
|
||||
asyncFetchAcquireRelease
|
||||
acquire release dispatch wait enqueue _state _flags _si requests =
|
||||
AsyncFetch $ \inner -> mask $ \restore -> do
|
||||
acquire release dispatch wait enqueue _state _flags _si =
|
||||
AsyncFetch $ \requests inner -> mask $ \restore -> do
|
||||
r1 <- tryWithRethrow acquire
|
||||
case r1 of
|
||||
Left err -> do restore inner; throwIO (err :: SomeException)
|
||||
|
@ -76,7 +76,18 @@ instance DataSourceName UserReq where
|
||||
dataSourceName _ = "UserDataSource"
|
||||
|
||||
instance DataSource u UserReq where
|
||||
fetch _state _flags _userEnv blockedFetches = SyncFetch $ do
|
||||
fetch _state _flags _userEnv = SyncFetch $ \blockedFetches -> do
|
||||
let
|
||||
allIdVars :: [ResultVar [Id]]
|
||||
allIdVars = [r | BlockedFetch GetAllIds r <- blockedFetches]
|
||||
|
||||
idStrings :: [String]
|
||||
idStrings = map show ids
|
||||
|
||||
ids :: [Id]
|
||||
vars :: [ResultVar Name]
|
||||
(ids, vars) = unzip
|
||||
[(userId, r) | BlockedFetch (GetNameById userId) r <- blockedFetches]
|
||||
|
||||
unless (null allIdVars) $ do
|
||||
allIds <- sql "select id from ids"
|
||||
@ -90,18 +101,6 @@ instance DataSource u UserReq where
|
||||
]
|
||||
mapM_ (uncurry putSuccess) (zip vars names)
|
||||
|
||||
where
|
||||
allIdVars :: [ResultVar [Id]]
|
||||
allIdVars = [r | BlockedFetch GetAllIds r <- blockedFetches]
|
||||
|
||||
idStrings :: [String]
|
||||
idStrings = map show ids
|
||||
|
||||
ids :: [Id]
|
||||
vars :: [ResultVar Name]
|
||||
(ids, vars) = unzip
|
||||
[(userId, r) | BlockedFetch (GetNameById userId) r <- blockedFetches]
|
||||
|
||||
-- Mock SQL API.
|
||||
|
||||
class SQLResult a where
|
||||
|
@ -58,6 +58,7 @@ library
|
||||
-- text 1.2.1.0 required for instance Binary Text
|
||||
text >= 1.2.1.0 && < 1.3,
|
||||
time >= 1.4 && < 1.9,
|
||||
stm == 2.4.*,
|
||||
transformers,
|
||||
unordered-containers == 0.2.*,
|
||||
vector >= 0.10 && <0.13
|
||||
@ -80,6 +81,7 @@ library
|
||||
default-language: Haskell2010
|
||||
|
||||
ghc-options:
|
||||
-O2 -fprof-auto
|
||||
-Wall
|
||||
|
||||
|
||||
@ -122,10 +124,12 @@ test-suite test
|
||||
CoreTests
|
||||
DataCacheTest
|
||||
ExampleDataSource
|
||||
FullyAsyncTest
|
||||
LoadCache
|
||||
MemoizationTests
|
||||
MockTAO
|
||||
ProfileTests
|
||||
SleepDataSource
|
||||
TestBadDataSource
|
||||
TestExampleDataSource
|
||||
TestTypes
|
||||
|
@ -42,8 +42,8 @@ ado3_ = do
|
||||
b' <- friendsOf =<< if null b then id4 else id3
|
||||
return (length (a' ++ b'))
|
||||
|
||||
tests = TestList
|
||||
[ TestLabel "ado1" $ TestCase ado1
|
||||
, TestLabel "ado2" $ TestCase ado2
|
||||
, TestLabel "ado3" $ TestCase ado3
|
||||
tests future = TestList
|
||||
[ TestLabel "ado1" $ TestCase (ado1 future)
|
||||
, TestLabel "ado2" $ TestCase (ado2 future)
|
||||
, TestLabel "ado3" $ TestCase (ado3 future)
|
||||
]
|
||||
|
@ -13,21 +13,24 @@ import ProfileTests
|
||||
#endif
|
||||
import MemoizationTests
|
||||
import TestBadDataSource
|
||||
import FullyAsyncTest
|
||||
|
||||
import Test.HUnit
|
||||
|
||||
allTests :: Test
|
||||
allTests = TestList
|
||||
[ TestLabel "ExampleDataSource" TestExampleDataSource.tests
|
||||
, TestLabel "BatchTests" BatchTests.tests
|
||||
, TestLabel "BatchTests-future" $ BatchTests.tests True
|
||||
, TestLabel "BatchTests-sync" $ BatchTests.tests False
|
||||
, TestLabel "CoreTests" CoreTests.tests
|
||||
, TestLabel "DataCacheTests" DataCacheTest.tests
|
||||
#if __GLASGOW_HASKELL__ >= 801
|
||||
, TestLabel "AdoTests" AdoTests.tests
|
||||
, TestLabel "AdoTests" $ AdoTests.tests False
|
||||
#endif
|
||||
#if __GLASGOW_HASKELL__ >= 710
|
||||
, TestLabel "ProfileTests" ProfileTests.tests
|
||||
#endif
|
||||
, TestLabel "MemoizationTests" MemoizationTests.tests
|
||||
, TestLabel "BadDataSourceTests" TestBadDataSource.tests
|
||||
, TestLabel "FullyAsyncTest" FullyAsyncTest.tests
|
||||
]
|
||||
|
@ -127,20 +127,20 @@ caching3_ = if nf id1 .> 0 then nf id1 + nf id2 + nf id3 else 0
|
||||
--
|
||||
-- Basic sanity check on data-cache re-use
|
||||
--
|
||||
cacheReuse = do
|
||||
env <- makeTestEnv
|
||||
cacheReuse future = do
|
||||
env <- makeTestEnv future
|
||||
expectRoundsWithEnv 2 12 batching7_ env
|
||||
|
||||
-- make a new env
|
||||
tao <- MockTAO.initGlobalState
|
||||
tao <- MockTAO.initGlobalState future
|
||||
let st = stateSet tao stateEmpty
|
||||
env2 <- initEnvWithData st testinput (caches env)
|
||||
|
||||
-- ensure no more data fetching rounds needed
|
||||
expectRoundsWithEnv 0 12 batching7_ env2
|
||||
|
||||
noCaching = do
|
||||
env <- makeTestEnv
|
||||
noCaching future = do
|
||||
env <- makeTestEnv future
|
||||
let env' = env{ flags = (flags env){caching = 0} }
|
||||
result <- runHaxl env' caching3_
|
||||
assertEqual "result" result 18
|
||||
@ -155,8 +155,8 @@ exceptionTest2 = expectRounds 1 [7..12] $ liftA2 (++)
|
||||
(withDefault [] (friendsOf 101))
|
||||
(withDefault [] (friendsOf 2))
|
||||
|
||||
deterministicExceptions = do
|
||||
env <- makeTestEnv
|
||||
deterministicExceptions future = do
|
||||
env <- makeTestEnv future
|
||||
let haxl =
|
||||
sequence [ do _ <- friendsOf =<< id1; throw (NotFound "xxx")
|
||||
, throw (NotFound "yyy")
|
||||
@ -176,8 +176,8 @@ deterministicExceptions = do
|
||||
Left (NotFound "xxx") -> True
|
||||
_ -> False
|
||||
|
||||
pOrTests = do
|
||||
env <- makeTestEnv
|
||||
pOrTests future = do
|
||||
env <- makeTestEnv future
|
||||
|
||||
-- Test semantics
|
||||
r <- runHaxl env $ do
|
||||
@ -224,8 +224,8 @@ pOrTests = do
|
||||
Left (NotFound _) -> True
|
||||
_ -> False
|
||||
|
||||
pAndTests = do
|
||||
env <- makeTestEnv
|
||||
pAndTests future = do
|
||||
env <- makeTestEnv future
|
||||
|
||||
-- Test semantics
|
||||
r <- runHaxl env $ do
|
||||
@ -273,24 +273,26 @@ pAndTests = do
|
||||
Left (NotFound _) -> True
|
||||
_ -> False
|
||||
|
||||
tests = TestList
|
||||
[ TestLabel "batching1" $ TestCase batching1
|
||||
, TestLabel "batching2" $ TestCase batching2
|
||||
, TestLabel "batching3" $ TestCase batching3
|
||||
, TestLabel "batching4" $ TestCase batching4
|
||||
, TestLabel "batching5" $ TestCase batching5
|
||||
, TestLabel "batching6" $ TestCase batching6
|
||||
, TestLabel "batching7" $ TestCase batching7
|
||||
, TestLabel "batching8" $ TestCase batching8
|
||||
, TestLabel "batching9" $ TestCase batching9
|
||||
, TestLabel "caching1" $ TestCase caching1
|
||||
, TestLabel "caching2" $ TestCase caching2
|
||||
, TestLabel "caching3" $ TestCase caching3
|
||||
, TestLabel "CacheReuse" $ TestCase cacheReuse
|
||||
, TestLabel "NoCaching" $ TestCase noCaching
|
||||
, TestLabel "exceptionTest1" $ TestCase exceptionTest1
|
||||
, TestLabel "exceptionTest2" $ TestCase exceptionTest2
|
||||
, TestLabel "deterministicExceptions" $ TestCase deterministicExceptions
|
||||
, TestLabel "pOrTest" $ TestCase pOrTests
|
||||
, TestLabel "pAndTest" $ TestCase pAndTests
|
||||
tests :: Bool -> Test
|
||||
tests future = TestList
|
||||
[ TestLabel "batching1" $ TestCase (batching1 future)
|
||||
, TestLabel "batching2" $ TestCase (batching2 future)
|
||||
, TestLabel "batching3" $ TestCase (batching3 future)
|
||||
, TestLabel "batching4" $ TestCase (batching4 future)
|
||||
, TestLabel "batching5" $ TestCase (batching5 future)
|
||||
, TestLabel "batching6" $ TestCase (batching6 future)
|
||||
, TestLabel "batching7" $ TestCase (batching7 future)
|
||||
, TestLabel "batching8" $ TestCase (batching8 future)
|
||||
, TestLabel "batching9" $ TestCase (batching9 future)
|
||||
, TestLabel "caching1" $ TestCase (caching1 future)
|
||||
, TestLabel "caching2" $ TestCase (caching2 future)
|
||||
, TestLabel "caching3" $ TestCase (caching3 future)
|
||||
, TestLabel "CacheReuse" $ TestCase (cacheReuse future)
|
||||
, TestLabel "NoCaching" $ TestCase (noCaching future)
|
||||
, TestLabel "exceptionTest1" $ TestCase (exceptionTest1 future)
|
||||
, TestLabel "exceptionTest2" $ TestCase (exceptionTest2 future)
|
||||
, TestLabel "deterministicExceptions" $
|
||||
TestCase (deterministicExceptions future)
|
||||
, TestLabel "pOrTest" $ TestCase (pOrTests future)
|
||||
, TestLabel "pAndTest" $ TestCase (pAndTests future)
|
||||
]
|
||||
|
@ -5,11 +5,11 @@
|
||||
module Bench where
|
||||
|
||||
import Haxl.Core.DataCache as DataCache
|
||||
import Haxl.Core.Types
|
||||
|
||||
import Prelude hiding (mapM)
|
||||
|
||||
import Data.Hashable
|
||||
import Data.IORef
|
||||
import Data.Time.Clock
|
||||
import Data.Traversable
|
||||
import Data.Typeable
|
||||
@ -36,12 +36,12 @@ main = do
|
||||
let
|
||||
f 0 !cache = return cache
|
||||
f !n !cache = do
|
||||
m <- newResult 0
|
||||
m <- newIORef 0
|
||||
f (n-1) (DataCache.insert (ReqInt n) m cache)
|
||||
--
|
||||
cache <- f n emptyDataCache
|
||||
let m = DataCache.lookup (ReqInt (n `div` 2)) cache
|
||||
print =<< mapM takeResult m
|
||||
print =<< mapM readIORef m
|
||||
t1 <- getCurrentTime
|
||||
printf "insert: %.2fs\n" (realToFrac (t1 `diffUTCTime` t0) :: Double)
|
||||
|
||||
|
@ -12,17 +12,29 @@ import Test.HUnit
|
||||
|
||||
import Data.Aeson
|
||||
import qualified Data.ByteString.Lazy.Char8 as BS
|
||||
import Data.List
|
||||
|
||||
import Control.Exception (Exception(..))
|
||||
|
||||
import ExampleDataSource
|
||||
|
||||
testEnv = do
|
||||
-- To use a data source, we need to initialize its state:
|
||||
exstate <- ExampleDataSource.initGlobalState
|
||||
|
||||
-- And create a StateStore object containing the states we need:
|
||||
let st = stateSet exstate stateEmpty
|
||||
|
||||
-- Create the Env:
|
||||
initEnv st ()
|
||||
|
||||
useless :: String -> GenHaxl u Bool
|
||||
useless _ = throw (NotFound "ha ha")
|
||||
|
||||
en = error "no env"
|
||||
|
||||
exceptions :: Assertion
|
||||
exceptions =
|
||||
do
|
||||
en <- emptyEnv ()
|
||||
a <- runHaxl en $ try (useless "input")
|
||||
assertBool "NotFound -> HaxlException" $
|
||||
isLeft (a :: Either HaxlException Bool)
|
||||
@ -72,6 +84,38 @@ exceptions =
|
||||
return True)
|
||||
`catch` \InternalError{} -> return False
|
||||
assertBool "catchIf2" (not e)
|
||||
|
||||
-- test tryToHaxlException
|
||||
e <- runHaxl en $ tryToHaxlException $ head []
|
||||
assertBool "tryToHaxlException1" $
|
||||
case e of
|
||||
Left ex | Just NonHaxlException{} <- fromException (toException ex)
|
||||
-> True
|
||||
_ -> False
|
||||
|
||||
env <- testEnv
|
||||
e <- runHaxl env $ tryToHaxlException $ do
|
||||
xs <- listWombats 3
|
||||
return $! length xs `quot` 0
|
||||
print e
|
||||
assertBool "tryToHaxlException1" $
|
||||
case e of
|
||||
Left ex | Just NonHaxlException{} <- fromException (toException ex)
|
||||
-> True
|
||||
_ -> False
|
||||
|
||||
env <- testEnv
|
||||
e <- runHaxl env $ mapM tryToHaxlException
|
||||
[ do xs <- listWombats 3; return $! length xs `quot` 0
|
||||
, head []
|
||||
]
|
||||
print e
|
||||
assertBool "tryToHaxlException2" $
|
||||
case e of
|
||||
[Left ex1, Left ex2]
|
||||
| "divide" `isInfixOf` show ex1
|
||||
, "head" `isInfixOf` show ex2 -> True
|
||||
_ -> False
|
||||
where
|
||||
isLeft Left{} = True
|
||||
isLeft _ = False
|
||||
@ -80,7 +124,9 @@ exceptions =
|
||||
-- This is mostly a compile test, to make sure all the plumbing
|
||||
-- makes the compiler happy.
|
||||
base :: (Exception a) => a -> IO HaxlException
|
||||
base e = runHaxl en $ throw e `catch` \x -> return x
|
||||
base e = do
|
||||
en <- emptyEnv ()
|
||||
runHaxl en $ throw e `catch` \x -> return x
|
||||
|
||||
printing :: Assertion
|
||||
printing = do
|
||||
@ -98,7 +144,27 @@ printing = do
|
||||
BS.putStrLn $ encode c
|
||||
|
||||
|
||||
withEnvTest :: Test
|
||||
withEnvTest = TestLabel "withEnvTest" $ TestCase $ do
|
||||
exstate <- ExampleDataSource.initGlobalState
|
||||
e <- initEnv (stateSet exstate stateEmpty) False
|
||||
b <- runHaxl e $ withEnv e { userEnv = True } $ env userEnv
|
||||
assertBool "withEnv1" b
|
||||
e <- initEnv (stateSet exstate stateEmpty) False
|
||||
b <- runHaxl e $ withEnv e { userEnv = True } $ do
|
||||
_ <- countAardvarks "aaa"
|
||||
env userEnv
|
||||
assertBool "withEnv2" b
|
||||
e <- initEnv (stateSet exstate stateEmpty) False
|
||||
b <- runHaxl e $ withEnv e { userEnv = True } $ do
|
||||
memo ("xxx" :: Text) $ do
|
||||
_ <- countAardvarks "aaa"
|
||||
env userEnv
|
||||
assertBool "withEnv3" b
|
||||
|
||||
|
||||
tests = TestList
|
||||
[ TestLabel "exceptions" $ TestCase exceptions,
|
||||
TestLabel "print_stuff" $ TestCase printing
|
||||
TestLabel "print_stuff" $ TestCase printing,
|
||||
TestLabel "withEnv" $ withEnvTest
|
||||
]
|
||||
|
@ -1,7 +1,8 @@
|
||||
{-# LANGUAGE StandaloneDeriving, GADTs, DeriveDataTypeable #-}
|
||||
module DataCacheTest (tests) where
|
||||
module DataCacheTest (tests, newResult, takeResult) where
|
||||
|
||||
import Haxl.Core.DataCache as DataCache
|
||||
import Haxl.Core.Monad
|
||||
import Haxl.Core
|
||||
|
||||
import Control.Exception
|
||||
@ -10,6 +11,7 @@ import Data.Traversable
|
||||
import Data.Typeable
|
||||
import Prelude hiding (mapM)
|
||||
import Test.HUnit
|
||||
import Data.IORef
|
||||
|
||||
data TestReq a where
|
||||
Req :: Int -> TestReq a -- polymorphic result
|
||||
@ -21,6 +23,16 @@ deriving instance Show (TestReq a)
|
||||
instance Hashable (TestReq a) where
|
||||
hashWithSalt salt (Req i) = hashWithSalt salt i
|
||||
|
||||
newResult :: a -> IO (IVar u a)
|
||||
newResult a = IVar <$> newIORef (IVarFull (Ok a))
|
||||
|
||||
takeResult :: IVar u a -> IO (ResultVal a)
|
||||
takeResult (IVar ref) = do
|
||||
e <- readIORef ref
|
||||
case e of
|
||||
IVarFull a -> return a
|
||||
_ -> error "takeResult"
|
||||
|
||||
|
||||
dcSoundnessTest :: Test
|
||||
dcSoundnessTest = TestLabel "DataCache soundness" $ TestCase $ do
|
||||
@ -35,25 +47,25 @@ dcSoundnessTest = TestLabel "DataCache soundness" $ TestCase $ do
|
||||
-- with a result of type String, we should get Nothing, not a crash.
|
||||
r <- mapM takeResult $ DataCache.lookup (Req 1) cache
|
||||
assertBool "dcSoundness1" $
|
||||
case r :: Maybe (Either SomeException String) of
|
||||
case r :: Maybe (ResultVal String) of
|
||||
Nothing -> True
|
||||
_something_else -> False
|
||||
|
||||
r <- mapM takeResult $ DataCache.lookup (Req 1) cache
|
||||
assertBool "dcSoundness2" $
|
||||
case r :: Maybe (Either SomeException Int) of
|
||||
Just (Right 1) -> True
|
||||
case r :: Maybe (ResultVal Int) of
|
||||
Just (Ok 1) -> True
|
||||
_something_else -> False
|
||||
|
||||
r <- mapM takeResult $ DataCache.lookup (Req 2) cache
|
||||
assertBool "dcSoundness3" $
|
||||
case r :: Maybe (Either SomeException String) of
|
||||
Just (Right "hello") -> True
|
||||
case r :: Maybe (ResultVal String) of
|
||||
Just (Ok "hello") -> True
|
||||
_something_else -> False
|
||||
|
||||
r <- mapM takeResult $ DataCache.lookup (Req 2) cache
|
||||
assertBool "dcSoundness4" $
|
||||
case r :: Maybe (Either SomeException Int) of
|
||||
case r :: Maybe (ResultVal Int) of
|
||||
Nothing -> True
|
||||
_something_else -> False
|
||||
|
||||
|
@ -117,10 +117,9 @@ initGlobalState = do
|
||||
exampleFetch :: State ExampleReq -- current state
|
||||
-> Flags -- tracing verbosity, etc.
|
||||
-> u -- user environment
|
||||
-> [BlockedFetch ExampleReq] -- requests to fetch
|
||||
-> PerformFetch -- tells the framework how to fetch
|
||||
-> PerformFetch ExampleReq -- tells the framework how to fetch
|
||||
|
||||
exampleFetch _state _flags _user bfs = SyncFetch $ mapM_ fetch1 bfs
|
||||
exampleFetch _state _flags _user = SyncFetch $ mapM_ fetch1
|
||||
|
||||
-- There are two ways a data source can fetch data: synchronously or
|
||||
-- asynchronously. See the type 'PerformFetch' in "Haxl.Core.Types" for
|
||||
@ -150,8 +149,8 @@ fetch1 (BlockedFetch (ListWombats a) r) =
|
||||
-- Normally a data source will provide some convenient wrappers for
|
||||
-- its requests:
|
||||
|
||||
countAardvarks :: String -> GenHaxl () Int
|
||||
countAardvarks :: String -> GenHaxl u Int
|
||||
countAardvarks str = dataFetch (CountAardvarks str)
|
||||
|
||||
listWombats :: Id -> GenHaxl () [Id]
|
||||
listWombats :: Id -> GenHaxl u [Id]
|
||||
listWombats i = dataFetch (ListWombats i)
|
||||
|
8
tests/FBMain.hs
Normal file
8
tests/FBMain.hs
Normal file
@ -0,0 +1,8 @@
|
||||
module FBMain (main) where
|
||||
|
||||
import Facebook.Init
|
||||
import TestRunner
|
||||
import AllTests
|
||||
|
||||
main :: IO ()
|
||||
main = withFacebookUnitTest $ testRunner $ allTests
|
59
tests/FullyAsyncTest.hs
Normal file
59
tests/FullyAsyncTest.hs
Normal file
@ -0,0 +1,59 @@
|
||||
module FullyAsyncTest where
|
||||
|
||||
import Haxl.Prelude as Haxl
|
||||
import Prelude()
|
||||
|
||||
import SleepDataSource
|
||||
|
||||
import Haxl.Core
|
||||
import Test.HUnit
|
||||
import Data.IORef
|
||||
import Haxl.Core.Monad (unsafeLiftIO)
|
||||
|
||||
tests :: Test
|
||||
tests = sleepTest
|
||||
|
||||
testEnv = do
|
||||
st <- SleepDataSource.initGlobalState
|
||||
env <- initEnv (stateSet st stateEmpty) ()
|
||||
return env { flags = (flags env) { report = 2 } }
|
||||
|
||||
sleepTest :: Test
|
||||
sleepTest = TestCase $ do
|
||||
env <- testEnv
|
||||
|
||||
ref <- newIORef ([] :: [Int])
|
||||
let tick n = unsafeLiftIO (modifyIORef ref (n:))
|
||||
|
||||
-- simulate running a selection of data fetches that complete at
|
||||
-- different times, overlapping them as much as possible.
|
||||
runHaxl env $
|
||||
sequence_
|
||||
[ sequence_ [sleep 100, sleep 400] `andThen` tick 5 -- A
|
||||
, sleep 100 `andThen` tick 2 `andThen` sleep 200 `andThen` tick 4 -- B
|
||||
, sleep 50 `andThen` tick 1 `andThen` sleep 150 `andThen` tick 3 -- C
|
||||
]
|
||||
|
||||
ys <- readIORef ref
|
||||
assertEqual "FullyAsyncTest: ordering" [1,2,3,4,5] (reverse ys)
|
||||
|
||||
stats <- readIORef (statsRef env)
|
||||
print stats
|
||||
assertEqual "FullyAsyncTest: stats" 5 (numFetches stats)
|
||||
|
||||
andThen :: GenHaxl u a -> GenHaxl u b -> GenHaxl u b
|
||||
andThen a b = a >>= \_ -> b
|
||||
|
||||
{-
|
||||
A B C
|
||||
50 | | tick 1
|
||||
100 | tick 2 |
|
||||
150 | | |
|
||||
200 | | tick 3
|
||||
250 | |
|
||||
300 | tick 4
|
||||
350 |
|
||||
400 |
|
||||
450 |
|
||||
500 tick 5
|
||||
-}
|
@ -6,6 +6,7 @@
|
||||
{-# LANGUAGE TypeFamilies #-}
|
||||
{-# LANGUAGE TypeSynonymInstances #-}
|
||||
{-# LANGUAGE FlexibleInstances #-}
|
||||
{-# LANGUAGE RecordWildCards #-}
|
||||
|
||||
module MockTAO (
|
||||
Id(..),
|
||||
@ -43,16 +44,18 @@ instance Hashable (TAOReq a) where
|
||||
hashWithSalt s (AssocRangeId2s a b) = hashWithSalt s (a,b)
|
||||
|
||||
instance StateKey TAOReq where
|
||||
data State TAOReq = TAOState {}
|
||||
data State TAOReq = TAOState { future :: Bool }
|
||||
|
||||
instance DataSourceName TAOReq where
|
||||
dataSourceName _ = "MockTAO"
|
||||
|
||||
instance DataSource UserEnv TAOReq where
|
||||
fetch _state _flags _user bfs = SyncFetch $ mapM_ doFetch bfs
|
||||
fetch TAOState{..} _flags _user
|
||||
| future = FutureFetch $ return . mapM_ doFetch
|
||||
| otherwise = SyncFetch $ mapM_ doFetch
|
||||
|
||||
initGlobalState :: IO (State TAOReq)
|
||||
initGlobalState = return TAOState {}
|
||||
initGlobalState :: Bool -> IO (State TAOReq)
|
||||
initGlobalState future = return TAOState { future=future }
|
||||
|
||||
doFetch :: BlockedFetch TAOReq -> IO ()
|
||||
doFetch (BlockedFetch req@(AssocRangeId2s a b) r) =
|
||||
|
@ -118,7 +118,8 @@ main = do
|
||||
]
|
||||
exitWith (ExitFailure 1)
|
||||
t1 <- getCurrentTime
|
||||
printf "%d reqs: %.2fs\n" n (realToFrac (t1 `diffUTCTime` t0) :: Double)
|
||||
printf "%10s: %10d reqs: %.2fs\n"
|
||||
test n (realToFrac (t1 `diffUTCTime` t0) :: Double)
|
||||
where
|
||||
-- can't use >>, it is aliased to *> and we want the real bind here
|
||||
andThen x y = x >>= const y
|
||||
|
@ -3,7 +3,6 @@
|
||||
module ProfileTests where
|
||||
|
||||
import Haxl.Prelude
|
||||
import Data.List
|
||||
|
||||
import Haxl.Core
|
||||
import Haxl.Core.Monad
|
||||
@ -12,6 +11,7 @@ import Test.HUnit
|
||||
|
||||
import Control.DeepSeq (force)
|
||||
import Control.Exception (evaluate)
|
||||
import Data.Aeson
|
||||
import Data.IORef
|
||||
import qualified Data.HashMap.Strict as HashMap
|
||||
import qualified Data.HashSet as HashSet
|
||||
@ -19,7 +19,7 @@ import qualified Data.HashSet as HashSet
|
||||
import TestUtils
|
||||
|
||||
mkProfilingEnv = do
|
||||
env <- makeTestEnv
|
||||
env <- makeTestEnv False
|
||||
return env { flags = (flags env) { report = 4 } }
|
||||
|
||||
collectsdata :: Assertion
|
||||
@ -29,17 +29,20 @@ collectsdata = do
|
||||
withLabel "bar" $
|
||||
withLabel "foo" $ do
|
||||
u <- env userEnv
|
||||
if length (intersect (HashMap.keys u) ["c"]) > 1
|
||||
then return 5
|
||||
else return (4::Int)
|
||||
-- do some non-trivial work that can't be lifted out
|
||||
case fromJSON <$> HashMap.lookup "A" u of
|
||||
Just (Success n) | sum [n .. 1000::Integer] > 0 -> return 5
|
||||
_otherwise -> return (4::Int)
|
||||
profData <- profile <$> readIORef (profRef e)
|
||||
assertEqual "has data" 3 $ HashMap.size profData
|
||||
assertBool "foo allocates" $
|
||||
case profileAllocs <$> HashMap.lookup "foo" profData of
|
||||
Just x -> x > 0
|
||||
Just x -> x > 10000
|
||||
Nothing -> False
|
||||
assertEqual "bar does not allocate" (Just 0) $
|
||||
profileAllocs <$> HashMap.lookup "bar" profData
|
||||
assertBool "bar does not allocate (much)" $
|
||||
case profileAllocs <$> HashMap.lookup "bar" profData of
|
||||
Just n -> n < 5000 -- getAllocationCounter can be off by +/- 4K
|
||||
_otherwise -> False
|
||||
assertEqual "foo's parent" (Just ["bar"]) $
|
||||
HashSet.toList . profileDeps <$> HashMap.lookup "foo" profData
|
||||
|
||||
|
59
tests/SleepDataSource.hs
Normal file
59
tests/SleepDataSource.hs
Normal file
@ -0,0 +1,59 @@
|
||||
{-# LANGUAGE DeriveDataTypeable #-}
|
||||
{-# LANGUAGE GADTs #-}
|
||||
{-# LANGUAGE MultiParamTypeClasses #-}
|
||||
{-# LANGUAGE OverloadedStrings #-}
|
||||
{-# LANGUAGE StandaloneDeriving #-}
|
||||
{-# LANGUAGE TypeFamilies #-}
|
||||
{-# LANGUAGE GeneralizedNewtypeDeriving #-}
|
||||
{-# LANGUAGE FlexibleInstances #-}
|
||||
|
||||
module SleepDataSource (
|
||||
-- * initialise the state
|
||||
initGlobalState,
|
||||
|
||||
-- * requests for this data source
|
||||
SleepReq(..),
|
||||
sleep,
|
||||
) where
|
||||
|
||||
import Haxl.Prelude
|
||||
import Prelude ()
|
||||
|
||||
import Haxl.Core
|
||||
|
||||
import Control.Monad hiding (mapM_)
|
||||
import Data.Typeable
|
||||
import Data.Hashable
|
||||
import Control.Concurrent
|
||||
|
||||
sleep :: Int -> GenHaxl u Int
|
||||
sleep n = dataFetch (Sleep n)
|
||||
|
||||
data SleepReq a where
|
||||
Sleep :: Int -> SleepReq Int
|
||||
deriving Typeable -- requests must be Typeable
|
||||
|
||||
deriving instance Eq (SleepReq a)
|
||||
deriving instance Show (SleepReq a)
|
||||
|
||||
instance ShowP SleepReq where showp = show
|
||||
|
||||
instance Hashable (SleepReq a) where
|
||||
hashWithSalt s (Sleep n) = hashWithSalt s n
|
||||
|
||||
instance StateKey SleepReq where
|
||||
data State SleepReq = ExampleState {}
|
||||
|
||||
instance DataSourceName SleepReq where
|
||||
dataSourceName _ = "SleepDataSource"
|
||||
|
||||
instance DataSource u SleepReq where
|
||||
fetch _state _flags _user = BackgroundFetch $ mapM_ fetch1
|
||||
schedulerHint _ = SubmitImmediately
|
||||
|
||||
initGlobalState :: IO (State SleepReq)
|
||||
initGlobalState = return ExampleState { }
|
||||
|
||||
fetch1 :: BlockedFetch SleepReq -> IO ()
|
||||
fetch1 (BlockedFetch (Sleep n) rvar) =
|
||||
void $ forkFinally (threadDelay (n*1000) >> return n) (putResult rvar)
|
@ -1,4 +1,5 @@
|
||||
{-# LANGUAGE CPP, OverloadedStrings, RebindableSyntax, MultiWayIf #-}
|
||||
{-# LANGUAGE RecordWildCards #-}
|
||||
module TestExampleDataSource (tests) where
|
||||
|
||||
import Haxl.Prelude as Haxl
|
||||
@ -7,7 +8,6 @@ import Prelude()
|
||||
import Haxl.Core.Monad (unsafeLiftIO)
|
||||
import Haxl.Core
|
||||
|
||||
import qualified Data.HashMap.Strict as HashMap
|
||||
import Test.HUnit
|
||||
import Data.IORef
|
||||
import Data.Maybe
|
||||
@ -26,7 +26,8 @@ testEnv = do
|
||||
let st = stateSet exstate stateEmpty
|
||||
|
||||
-- Create the Env:
|
||||
initEnv st ()
|
||||
env <- initEnv st ()
|
||||
return env{ flags = (flags env){ report = 2 } }
|
||||
|
||||
|
||||
tests = TestList [
|
||||
@ -34,9 +35,12 @@ tests = TestList [
|
||||
TestLabel "orderTest" orderTest,
|
||||
TestLabel "preCacheTest" preCacheTest,
|
||||
TestLabel "cachedComputationTest" cachedComputationTest,
|
||||
TestLabel "cacheResultTest" cacheResultTest,
|
||||
TestLabel "memoTest" memoTest,
|
||||
TestLabel "dataSourceExceptionTest" dataSourceExceptionTest,
|
||||
TestLabel "dumpCacheAsHaskell" dumpCacheTest]
|
||||
TestLabel "dumpCacheAsHaskell" dumpCacheTest,
|
||||
TestLabel "fetchError" fetchError
|
||||
]
|
||||
|
||||
-- Let's test ExampleDataSource.
|
||||
|
||||
@ -52,14 +56,15 @@ exampleTest = TestCase $ do
|
||||
|
||||
-- Should be just one fetching round:
|
||||
Stats stats <- readIORef (statsRef env)
|
||||
putStrLn (ppStats (Stats stats))
|
||||
assertEqual "rounds" 1 (length stats)
|
||||
|
||||
-- With two fetches:
|
||||
assertBool "reqs" $
|
||||
if | RoundStats { roundDataSources = m } : _ <- stats,
|
||||
Just (DataSourceRoundStats { dataSourceFetches = 2 })
|
||||
<- HashMap.lookup "ExampleDataSource" m -> True
|
||||
| otherwise -> False
|
||||
case stats of
|
||||
[FetchStats{..}] ->
|
||||
fetchDataSource == "ExampleDataSource" && fetchBatchSize == 2
|
||||
_otherwise -> False
|
||||
|
||||
-- Test side-effect ordering
|
||||
|
||||
@ -97,8 +102,8 @@ preCacheTest = TestCase $ do
|
||||
|
||||
x <- runHaxl env $ do
|
||||
cacheRequest (CountAardvarks "xxx") (Right 3)
|
||||
cacheRequest (ListWombats 100) (Right [1,2,3])
|
||||
countAardvarks "xxx" + (length <$> listWombats 100)
|
||||
cacheRequest (ListWombats 1000000) (Right [1,2,3])
|
||||
countAardvarks "xxx" + (length <$> listWombats 1000000)
|
||||
assertEqual "preCacheTest1" x (3 + 3)
|
||||
|
||||
y <- Control.Exception.try $ runHaxl env $ do
|
||||
@ -126,6 +131,16 @@ cachedComputationTest = TestCase $ do
|
||||
stats <- readIORef (statsRef env)
|
||||
assertEqual "fetches" 3 (numFetches stats)
|
||||
|
||||
cacheResultTest = TestCase $ do
|
||||
env <- testEnv
|
||||
ref <- newIORef 0
|
||||
let request = cacheResult (CountAardvarks "ababa") $ do
|
||||
modifyIORef ref (+1)
|
||||
readIORef ref
|
||||
r <- runHaxl env $ (+) <$> request <*> request
|
||||
assertEqual "cacheResult" 2 r
|
||||
|
||||
|
||||
-- Pretend CountAardvarks is a request computed by some Haxl code
|
||||
memoTest = TestCase $ do
|
||||
env <- testEnv
|
||||
@ -143,6 +158,18 @@ memoTest = TestCase $ do
|
||||
stats <- readIORef (statsRef env)
|
||||
assertEqual "fetches" 3 (numFetches stats)
|
||||
|
||||
-- Test that the FetchError gets returned properly, and that we have
|
||||
-- a failure logged in the stats.
|
||||
fetchError = TestCase $ do
|
||||
env <- testEnv
|
||||
r <- runHaxl env $ Haxl.try $
|
||||
(++) <$> listWombats 1000000 <*> listWombats 1000001
|
||||
assertBool "fetchError1" $ case r of
|
||||
Left FetchError{} -> True
|
||||
Right _ -> False
|
||||
Stats stats <- readIORef (statsRef env)
|
||||
assertEqual "fetchError2" 2 (sum [ fetchFailures | FetchStats{..} <- stats ])
|
||||
|
||||
dataSourceExceptionTest = TestCase $ do
|
||||
env <- testEnv
|
||||
|
||||
|
@ -40,11 +40,12 @@ id3 = lookupInput "C"
|
||||
id4 :: Haxl Id
|
||||
id4 = lookupInput "D"
|
||||
|
||||
makeTestEnv :: IO (Env UserEnv)
|
||||
makeTestEnv = do
|
||||
tao <- MockTAO.initGlobalState
|
||||
makeTestEnv :: Bool -> IO (Env UserEnv)
|
||||
makeTestEnv future = do
|
||||
tao <- MockTAO.initGlobalState future
|
||||
let st = stateSet tao stateEmpty
|
||||
initEnv st testinput
|
||||
env <- initEnv st testinput
|
||||
return env { flags = (flags env) { report = 2 } }
|
||||
|
||||
expectRoundsWithEnv
|
||||
:: (Eq a, Show a) => Int -> a -> Haxl a -> Env UserEnv -> Assertion
|
||||
@ -54,14 +55,14 @@ expectRoundsWithEnv n result haxl env = do
|
||||
stats <- readIORef (statsRef env)
|
||||
assertEqual "rounds" n (numRounds stats)
|
||||
|
||||
expectRounds :: (Eq a, Show a) => Int -> a -> Haxl a -> Assertion
|
||||
expectRounds n result haxl = do
|
||||
env <- makeTestEnv
|
||||
expectRounds :: (Eq a, Show a) => Int -> a -> Haxl a -> Bool -> Assertion
|
||||
expectRounds n result haxl future = do
|
||||
env <- makeTestEnv future
|
||||
expectRoundsWithEnv n result haxl env
|
||||
|
||||
expectFetches :: (Eq a, Show a) => Int -> Haxl a -> Assertion
|
||||
expectFetches n haxl = do
|
||||
env <- makeTestEnv
|
||||
expectFetches :: (Eq a, Show a) => Int -> Haxl a -> Bool -> Assertion
|
||||
expectFetches n haxl future = do
|
||||
env <- makeTestEnv future
|
||||
_ <- runHaxl env haxl
|
||||
stats <- readIORef (statsRef env)
|
||||
assertEqual "fetches" n (numFetches stats)
|
||||
|
Loading…
Reference in New Issue
Block a user