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:
Simon Marlow 2017-10-03 00:14:24 -07:00 committed by Facebook Github Bot
parent 762d110041
commit b67f7f6370
23 changed files with 1462 additions and 792 deletions

View File

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

View File

@ -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))

View File

@ -53,6 +53,7 @@ module Haxl.Core.Exception (
-- ** Internal exceptions
CriticalError(..),
DataSourceError(..),
NonHaxlException(..),
-- ** Logic exceptions
NotFound(..),

File diff suppressed because it is too large Load Diff

View File

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

View File

@ -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)

View File

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

View File

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

View File

@ -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)
]

View File

@ -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
]

View File

@ -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)
]

View File

@ -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)

View File

@ -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
]

View File

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

View File

@ -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
View 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
View 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
-}

View File

@ -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) =

View File

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

View File

@ -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
View 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)

View File

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

View File

@ -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)