mirror of
https://github.com/hasura/graphql-engine.git
synced 2024-11-14 05:45:24 +03:00
multiplexed subscriptions (#1934)
* add types to represent unparsed http gql requests This will help when we add caching of frequently used ASTs * query plan caching * move livequery to execute * add multiplexed module * session variable can be customised depending on the context Previously the value was always "current_setting('hasura.user')" * get rid of typemap requirement in reusable plan * subscriptions are multiplexed when possible * use lazytx for introspection to avoid acquiring a pg connection * refactor to make execute a completely decoupled module * don't issue a transaction for a query * don't use current setting for explained sql * move postgres related types to a different module * validate variableValues on postgres before multiplexing subs * don't user current_setting for queries over ws * plan_cache is only visible when developer flag is enabled * introduce 'batch size' when multiplexing subscriptions * bump stackage to 13.16 * fix schema_stitching test case error code * store hashes instead of actual responses for subscriptions * internal api to dump subscriptions state * remove PlanCache from SchemaCacheRef * allow live query options to be configured on server startup * capture metrics for multiplexed subscriptions * more metrics captured for multiplexed subs * switch to tvar based hashmap for faster snapshotting * livequery modules do not expose internal details * fix typo in live query env vars * switch to hasura's pg-client-hs
This commit is contained in:
parent
393110420d
commit
ce243f5899
@ -22,6 +22,11 @@ flag developer
|
||||
default: False
|
||||
manual: True
|
||||
|
||||
flag profile
|
||||
description: compile with profile options set
|
||||
default: False
|
||||
manual: True
|
||||
|
||||
flag local-console
|
||||
description: embeds resources needed by console and serves them at /static/
|
||||
default: False
|
||||
@ -138,9 +143,14 @@ library
|
||||
-- pretty printer
|
||||
, ansi-wl-pprint
|
||||
|
||||
-- metrics in multiplexed subs
|
||||
, ekg-core
|
||||
|
||||
exposed-modules: Hasura.Prelude
|
||||
, Hasura.Logging
|
||||
, Hasura.EncJSON
|
||||
, Hasura.Db
|
||||
, Hasura.Cache
|
||||
|
||||
, Hasura.Server.App
|
||||
, Hasura.Server.Auth
|
||||
@ -210,8 +220,13 @@ library
|
||||
, Hasura.GraphQL.Validate.InputValue
|
||||
, Hasura.GraphQL.Explain
|
||||
, Hasura.GraphQL.Execute
|
||||
, Hasura.GraphQL.Execute.Plan
|
||||
, Hasura.GraphQL.Execute.Query
|
||||
, Hasura.GraphQL.Execute.LiveQuery
|
||||
, Hasura.GraphQL.Execute.LiveQuery.Types
|
||||
, Hasura.GraphQL.Execute.LiveQuery.Multiplexed
|
||||
, Hasura.GraphQL.Execute.LiveQuery.Fallback
|
||||
, Hasura.GraphQL.Resolve
|
||||
, Hasura.GraphQL.Resolve.LiveQuery
|
||||
, Hasura.GraphQL.Resolve.BoolExp
|
||||
, Hasura.GraphQL.Resolve.Context
|
||||
, Hasura.GraphQL.Resolve.InputValue
|
||||
@ -229,6 +244,7 @@ library
|
||||
, Hasura.HTTP
|
||||
|
||||
, Data.Text.Extended
|
||||
, Data.Aeson.Extended
|
||||
, Data.Sequence.NonEmpty
|
||||
, Data.TByteString
|
||||
, Data.HashMap.Strict.InsOrd.Extended
|
||||
@ -270,8 +286,10 @@ library
|
||||
NoImplicitPrelude
|
||||
|
||||
|
||||
if flag(developer)
|
||||
if flag(profile)
|
||||
ghc-prof-options: -rtsopts -fprof-auto -fno-prof-count-entries
|
||||
if flag(developer)
|
||||
cpp-options: -DInternalAPIs -DLocalConsole
|
||||
if flag(local-console)
|
||||
cpp-options: -DLocalConsole
|
||||
|
||||
|
@ -21,6 +21,7 @@ import qualified Network.HTTP.Client as HTTP
|
||||
import qualified Network.HTTP.Client.TLS as HTTP
|
||||
import qualified Network.Wai.Handler.Warp as Warp
|
||||
|
||||
import Hasura.Db
|
||||
import Hasura.Events.Lib
|
||||
import Hasura.Logging
|
||||
import Hasura.Prelude
|
||||
@ -76,6 +77,9 @@ parseHGECommand =
|
||||
<*> parseWsReadCookie
|
||||
<*> parseStringifyNum
|
||||
<*> parseEnabledAPIs
|
||||
<*> parseMxRefetchInt
|
||||
<*> parseMxBatchSize
|
||||
<*> parseFallbackRefetchInt
|
||||
|
||||
|
||||
parseArgs :: IO HGEOptions
|
||||
@ -112,8 +116,9 @@ main = do
|
||||
let logger = mkLogger loggerCtx
|
||||
pgLogger = mkPGLogger logger
|
||||
case hgeCmd of
|
||||
HCServe so@(ServeOptions port host cp isoL mAdminSecret mAuthHook mJwtSecret
|
||||
mUnAuthRole corsCfg enableConsole enableTelemetry strfyNum enabledAPIs) -> do
|
||||
HCServe so@(ServeOptions port host cp isoL mAdminSecret mAuthHook
|
||||
mJwtSecret mUnAuthRole corsCfg enableConsole
|
||||
enableTelemetry strfyNum enabledAPIs lqOpts) -> do
|
||||
-- log serve options
|
||||
unLogger logger $ serveOptsToLog so
|
||||
hloggerCtx <- mkLoggerCtx $ defaultLoggerSettings False
|
||||
@ -137,7 +142,7 @@ main = do
|
||||
|
||||
(app, cacheRef, cacheInitTime) <-
|
||||
mkWaiApp isoL loggerCtx strfyNum pool httpManager am
|
||||
corsCfg enableConsole enableTelemetry instanceId enabledAPIs
|
||||
corsCfg enableConsole enableTelemetry instanceId enabledAPIs lqOpts
|
||||
|
||||
-- start a background thread for schema sync
|
||||
startSchemaSync strfyNum pool logger httpManager
|
||||
@ -154,7 +159,8 @@ main = do
|
||||
let scRef = _scrCache cacheRef
|
||||
unLogger logger $
|
||||
mkGenericStrLog "event_triggers" "starting workers"
|
||||
void $ C.forkIO $ processEventQueue hloggerCtx logEnvHeaders httpManager pool scRef eventEngineCtx
|
||||
void $ C.forkIO $ processEventQueue hloggerCtx logEnvHeaders
|
||||
httpManager pool scRef eventEngineCtx
|
||||
|
||||
-- start a background thread to check for updates
|
||||
void $ C.forkIO $ checkForUpdates loggerCtx httpManager
|
||||
@ -194,7 +200,7 @@ main = do
|
||||
runAsAdmin pgLogger ci httpManager m = do
|
||||
pool <- getMinimalPool pgLogger ci
|
||||
res <- runExceptT $ peelRun emptySchemaCache adminUserInfo
|
||||
httpManager False pool Q.Serializable m
|
||||
httpManager False (PGExecCtx pool Q.Serializable) m
|
||||
return $ fmap fst res
|
||||
|
||||
procConnInfo rci =
|
||||
|
13
server/src-lib/Data/Aeson/Extended.hs
Normal file
13
server/src-lib/Data/Aeson/Extended.hs
Normal file
@ -0,0 +1,13 @@
|
||||
module Data.Aeson.Extended
|
||||
( module J
|
||||
, encodeToStrictText
|
||||
) where
|
||||
|
||||
import Hasura.Prelude
|
||||
|
||||
import Data.Aeson as J
|
||||
import qualified Data.Aeson.Text as JT
|
||||
import qualified Data.Text.Lazy as LT
|
||||
|
||||
encodeToStrictText :: (ToJSON a) => a -> Text
|
||||
encodeToStrictText = LT.toStrict . JT.encodeToLazyText
|
88
server/src-lib/Hasura/Cache.hs
Normal file
88
server/src-lib/Hasura/Cache.hs
Normal file
@ -0,0 +1,88 @@
|
||||
module Hasura.Cache
|
||||
( UnboundedCache
|
||||
, initCache
|
||||
, clearCache
|
||||
, mapCache
|
||||
, insert
|
||||
, lookup
|
||||
) where
|
||||
|
||||
import Control.Concurrent (getNumCapabilities, myThreadId,
|
||||
threadCapability)
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Data.IORef as IORef
|
||||
import qualified Data.Vector as V
|
||||
|
||||
import Hasura.Prelude hiding (lookup)
|
||||
|
||||
newtype LocalCacheRef k v = LocalCacheRef (IORef.IORef (Map.HashMap k v))
|
||||
|
||||
mapLocalCacheRef
|
||||
:: ((k, v) -> a) -> LocalCacheRef k v -> IO [a]
|
||||
mapLocalCacheRef f (LocalCacheRef ioRef) =
|
||||
map f . Map.toList <$> IORef.readIORef ioRef
|
||||
|
||||
-- | Create a new LC cache of the given size.
|
||||
initLocalCache :: IO (LocalCacheRef k v)
|
||||
initLocalCache = LocalCacheRef <$> IORef.newIORef Map.empty
|
||||
|
||||
clearIO :: LocalCacheRef k v -> IO ()
|
||||
clearIO (LocalCacheRef ref)=
|
||||
IORef.atomicModifyIORef' ref $ const (Map.empty, ())
|
||||
|
||||
-- | Return the cached result of the action or, in the case of a cache
|
||||
-- miss, execute the action and insertLocal it in the cache.
|
||||
lookupIO :: (Hashable k, Eq k) => LocalCacheRef k v -> k -> IO (Maybe v)
|
||||
lookupIO (LocalCacheRef ref) k =
|
||||
Map.lookup k <$> IORef.readIORef ref
|
||||
|
||||
insertIO :: (Hashable k, Eq k) => LocalCacheRef k v -> k -> v -> IO ()
|
||||
insertIO (LocalCacheRef ref) k v =
|
||||
IORef.atomicModifyIORef' ref $ \c -> (Map.insert k v c, ())
|
||||
|
||||
-- | Using a stripe of multiple handles can improve the performance in
|
||||
-- the case of concurrent accesses since several handles can be
|
||||
-- accessed in parallel.
|
||||
newtype UnboundedCache k v = UnboundedCache (V.Vector (LocalCacheRef k v))
|
||||
|
||||
mapCache
|
||||
:: ((k, v) -> a) -> UnboundedCache k v -> IO [[a]]
|
||||
mapCache f (UnboundedCache localCaches) =
|
||||
mapM (mapLocalCacheRef f) $ V.toList localCaches
|
||||
|
||||
-- | Create a new 'StripedHandle' with the given number of stripes and
|
||||
-- the given capacity for each stripe.
|
||||
initCache :: IO (UnboundedCache k v)
|
||||
initCache = do
|
||||
capabilities <- getNumCapabilities
|
||||
UnboundedCache <$> V.replicateM capabilities initLocalCache
|
||||
|
||||
clearCache :: UnboundedCache k v -> IO ()
|
||||
clearCache (UnboundedCache caches) =
|
||||
V.mapM_ clearIO caches
|
||||
|
||||
{-# INLINE getLocal #-}
|
||||
getLocal :: UnboundedCache k v -> IO (LocalCacheRef k v)
|
||||
getLocal (UnboundedCache handles) = do
|
||||
|
||||
(i, _) <- myThreadId >>= threadCapability
|
||||
|
||||
-- The number of capability could be dynamically changed.
|
||||
-- So, let's check the upper boundary of the vector
|
||||
let lim = V.length handles
|
||||
j | i < lim = i
|
||||
| otherwise = i `mod` lim
|
||||
|
||||
return $ handles V.! j
|
||||
|
||||
-- | Striped version of 'cached'.
|
||||
insert
|
||||
:: (Hashable k, Eq k) => UnboundedCache k v -> k -> v -> IO ()
|
||||
insert striped k v = do
|
||||
localHandle <- getLocal striped
|
||||
insertIO localHandle k v
|
||||
|
||||
lookup :: (Hashable k, Eq k) => UnboundedCache k v -> k -> IO (Maybe v)
|
||||
lookup striped k = do
|
||||
localHandle <- getLocal striped
|
||||
lookupIO localHandle k
|
124
server/src-lib/Hasura/Db.hs
Normal file
124
server/src-lib/Hasura/Db.hs
Normal file
@ -0,0 +1,124 @@
|
||||
-- A module for postgres execution related types and operations
|
||||
|
||||
module Hasura.Db
|
||||
( MonadTx(..)
|
||||
, LazyTx
|
||||
|
||||
, PGExecCtx(..)
|
||||
, runLazyTx
|
||||
, runLazyTx'
|
||||
, withUserInfo
|
||||
|
||||
, RespTx
|
||||
, LazyRespTx
|
||||
, defaultTxErrorHandler
|
||||
) where
|
||||
|
||||
import qualified Data.Aeson.Extended as J
|
||||
import qualified Database.PG.Query as Q
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types.Error
|
||||
import Hasura.RQL.Types.Permission
|
||||
import Hasura.SQL.Types
|
||||
|
||||
data PGExecCtx
|
||||
= PGExecCtx
|
||||
{ _pecPool :: !Q.PGPool
|
||||
, _pecTxIsolation :: !Q.TxIsolation
|
||||
}
|
||||
|
||||
class (MonadError QErr m) => MonadTx m where
|
||||
liftTx :: Q.TxE QErr a -> m a
|
||||
|
||||
instance (MonadTx m) => MonadTx (StateT s m) where
|
||||
liftTx = lift . liftTx
|
||||
|
||||
instance (MonadTx m) => MonadTx (ReaderT s m) where
|
||||
liftTx = lift . liftTx
|
||||
|
||||
data LazyTx e a
|
||||
= LTErr !e
|
||||
| LTNoTx !a
|
||||
| LTTx !(Q.TxE e a)
|
||||
|
||||
lazyTxToQTx :: LazyTx e a -> Q.TxE e a
|
||||
lazyTxToQTx = \case
|
||||
LTErr e -> throwError e
|
||||
LTNoTx r -> return r
|
||||
LTTx tx -> tx
|
||||
|
||||
runLazyTx
|
||||
:: PGExecCtx
|
||||
-> LazyTx QErr a -> ExceptT QErr IO a
|
||||
runLazyTx (PGExecCtx pgPool txIso) = \case
|
||||
LTErr e -> throwError e
|
||||
LTNoTx a -> return a
|
||||
LTTx tx -> Q.runTx pgPool (txIso, Nothing) tx
|
||||
|
||||
runLazyTx'
|
||||
:: PGExecCtx -> LazyTx QErr a -> ExceptT QErr IO a
|
||||
runLazyTx' (PGExecCtx pgPool _) = \case
|
||||
LTErr e -> throwError e
|
||||
LTNoTx a -> return a
|
||||
LTTx tx -> Q.runTx' pgPool tx
|
||||
|
||||
type RespTx = Q.TxE QErr EncJSON
|
||||
type LazyRespTx = LazyTx QErr EncJSON
|
||||
|
||||
setHeadersTx :: UserVars -> Q.TxE QErr ()
|
||||
setHeadersTx uVars =
|
||||
Q.unitQE defaultTxErrorHandler setSess () False
|
||||
where
|
||||
setSess = Q.fromText $
|
||||
"SET LOCAL \"hasura.user\" = " <>
|
||||
pgFmtLit (J.encodeToStrictText uVars)
|
||||
|
||||
defaultTxErrorHandler :: Q.PGTxErr -> QErr
|
||||
defaultTxErrorHandler txe =
|
||||
let e = internalError "postgres query error"
|
||||
in e {qeInternal = Just $ J.toJSON txe}
|
||||
|
||||
withUserInfo :: UserInfo -> LazyTx QErr a -> LazyTx QErr a
|
||||
withUserInfo uInfo = \case
|
||||
LTErr e -> LTErr e
|
||||
LTNoTx a -> LTNoTx a
|
||||
LTTx tx -> LTTx $ setHeadersTx (userVars uInfo) >> tx
|
||||
|
||||
instance Functor (LazyTx e) where
|
||||
fmap f = \case
|
||||
LTErr e -> LTErr e
|
||||
LTNoTx a -> LTNoTx $ f a
|
||||
LTTx tx -> LTTx $ fmap f tx
|
||||
|
||||
instance Applicative (LazyTx e) where
|
||||
pure = LTNoTx
|
||||
|
||||
LTErr e <*> _ = LTErr e
|
||||
LTNoTx f <*> r = fmap f r
|
||||
LTTx _ <*> LTErr e = LTErr e
|
||||
LTTx txf <*> LTNoTx a = LTTx $ txf <*> pure a
|
||||
LTTx txf <*> LTTx tx = LTTx $ txf <*> tx
|
||||
|
||||
instance Monad (LazyTx e) where
|
||||
LTErr e >>= _ = LTErr e
|
||||
LTNoTx a >>= f = f a
|
||||
LTTx txa >>= f =
|
||||
LTTx $ txa >>= lazyTxToQTx . f
|
||||
|
||||
instance MonadError e (LazyTx e) where
|
||||
throwError = LTErr
|
||||
LTErr e `catchError` f = f e
|
||||
LTNoTx a `catchError` _ = LTNoTx a
|
||||
LTTx txe `catchError` f =
|
||||
LTTx $ txe `catchError` (lazyTxToQTx . f)
|
||||
|
||||
instance MonadTx (LazyTx QErr) where
|
||||
liftTx = LTTx
|
||||
|
||||
instance MonadTx (Q.TxE QErr) where
|
||||
liftTx = id
|
||||
|
||||
instance MonadIO (LazyTx QErr) where
|
||||
liftIO = LTTx . liftIO
|
@ -21,6 +21,7 @@ import qualified Data.ByteString as B
|
||||
import qualified Data.ByteString.Builder as BB
|
||||
import qualified Data.ByteString.Lazy as BL
|
||||
import qualified Data.Text.Encoding as TE
|
||||
import qualified Database.PG.Query as Q
|
||||
|
||||
-- encoded json
|
||||
-- TODO: can be improved with gadts capturing bytestring, lazybytestring
|
||||
@ -29,6 +30,9 @@ newtype EncJSON
|
||||
= EncJSON { unEncJSON :: BB.Builder }
|
||||
deriving (Semigroup, Monoid, IsString)
|
||||
|
||||
instance Q.FromCol EncJSON where
|
||||
fromCol = fmap encJFromBS . Q.fromCol
|
||||
|
||||
encJToLBS :: EncJSON -> BL.ByteString
|
||||
encJToLBS = BB.toLazyByteString . unEncJSON
|
||||
{-# INLINE encJToLBS #-}
|
||||
|
@ -48,7 +48,8 @@ invocationVersion = "2"
|
||||
|
||||
type LogEnvHeaders = Bool
|
||||
|
||||
type CacheRef = IORef SchemaCache
|
||||
newtype CacheRef
|
||||
= CacheRef { unCacheRef :: IORef (SchemaCache, SchemaCacheVer) }
|
||||
|
||||
newtype EventInternalErr
|
||||
= EventInternalErr QErr
|
||||
@ -170,14 +171,16 @@ initEventEngineCtx maxT fetchI = do
|
||||
return $ EventEngineCtx q c maxT fetchI
|
||||
|
||||
processEventQueue
|
||||
:: L.LoggerCtx -> LogEnvHeaders -> HTTP.Manager-> Q.PGPool -> CacheRef -> EventEngineCtx
|
||||
:: L.LoggerCtx -> LogEnvHeaders -> HTTP.Manager-> Q.PGPool
|
||||
-> IORef (SchemaCache, SchemaCacheVer) -> EventEngineCtx
|
||||
-> IO ()
|
||||
processEventQueue logctx logenv httpMgr pool cacheRef eectx = do
|
||||
threads <- mapM async [fetchThread , consumeThread]
|
||||
threads <- mapM async [fetchThread, consumeThread]
|
||||
void $ waitAny threads
|
||||
where
|
||||
fetchThread = pushEvents (mkHLogger logctx) pool eectx
|
||||
consumeThread = consumeEvents (mkHLogger logctx) logenv httpMgr pool cacheRef eectx
|
||||
consumeThread = consumeEvents (mkHLogger logctx)
|
||||
logenv httpMgr pool (CacheRef cacheRef) eectx
|
||||
|
||||
pushEvents
|
||||
:: HLogger -> Q.PGPool -> EventEngineCtx -> IO ()
|
||||
@ -209,7 +212,7 @@ processEvent
|
||||
=> LogEnvHeaders -> Q.PGPool -> Event -> m ()
|
||||
processEvent logenv pool e = do
|
||||
cacheRef <- asks getter
|
||||
cache <- liftIO $ readIORef cacheRef
|
||||
cache <- fmap fst $ liftIO $ readIORef $ unCacheRef cacheRef
|
||||
let meti = getEventTriggerInfoFromEvent cache e
|
||||
case meti of
|
||||
Nothing -> do
|
||||
|
@ -28,7 +28,7 @@ data SelOpCtx
|
||||
= SelOpCtx
|
||||
{ _socTable :: !QualifiedTable
|
||||
, _socHeaders :: ![T.Text]
|
||||
, _socFilter :: !AnnBoolExpSQL
|
||||
, _socFilter :: !AnnBoolExpPartialSQL
|
||||
, _socLimit :: !(Maybe Int)
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -36,7 +36,7 @@ data SelPkOpCtx
|
||||
= SelPkOpCtx
|
||||
{ _spocTable :: !QualifiedTable
|
||||
, _spocHeaders :: ![T.Text]
|
||||
, _spocFilter :: !AnnBoolExpSQL
|
||||
, _spocFilter :: !AnnBoolExpPartialSQL
|
||||
, _spocArgMap :: !PGColArgMap
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -44,7 +44,7 @@ data FuncQOpCtx
|
||||
= FuncQOpCtx
|
||||
{ _fqocTable :: !QualifiedTable
|
||||
, _fqocHeaders :: ![T.Text]
|
||||
, _fqocFilter :: !AnnBoolExpSQL
|
||||
, _fqocFilter :: !AnnBoolExpPartialSQL
|
||||
, _fqocLimit :: !(Maybe Int)
|
||||
, _fqocFunction :: !QualifiedFunction
|
||||
, _fqocArgs :: !FuncArgSeq
|
||||
@ -54,8 +54,8 @@ data UpdOpCtx
|
||||
= UpdOpCtx
|
||||
{ _uocTable :: !QualifiedTable
|
||||
, _uocHeaders :: ![T.Text]
|
||||
, _uocFilter :: !AnnBoolExpSQL
|
||||
, _uocPresetCols :: !PreSetCols
|
||||
, _uocFilter :: !AnnBoolExpPartialSQL
|
||||
, _uocPresetCols :: !PreSetColsPartial
|
||||
, _uocAllCols :: ![PGColInfo]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -63,7 +63,7 @@ data DelOpCtx
|
||||
= DelOpCtx
|
||||
{ _docTable :: !QualifiedTable
|
||||
, _docHeaders :: ![T.Text]
|
||||
, _docFilter :: !AnnBoolExpSQL
|
||||
, _docFilter :: !AnnBoolExpPartialSQL
|
||||
, _docAllCols :: ![PGColInfo]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
|
@ -1,11 +1,23 @@
|
||||
module Hasura.GraphQL.Execute
|
||||
( GQExecPlan(..)
|
||||
, getExecPlan
|
||||
|
||||
, ExecPlanPartial
|
||||
, getExecPlanPartial
|
||||
|
||||
, ExecOp(..)
|
||||
, ExecPlanResolved
|
||||
, getResolvedExecPlan
|
||||
, execRemoteGQ
|
||||
|
||||
, EP.PlanCache
|
||||
, EP.initPlanCache
|
||||
, EP.clearPlanCache
|
||||
, EP.dumpPlanCache
|
||||
) where
|
||||
|
||||
import Control.Exception (try)
|
||||
import Control.Lens
|
||||
import Data.Has
|
||||
|
||||
import qualified Data.ByteString.Lazy as BL
|
||||
import qualified Data.CaseInsensitive as CI
|
||||
@ -19,29 +31,80 @@ import qualified Network.HTTP.Types as N
|
||||
import qualified Network.Wreq as Wreq
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Context
|
||||
import Hasura.GraphQL.Resolve.Context
|
||||
import Hasura.GraphQL.Schema
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.GraphQL.Validate.Types
|
||||
import Hasura.HTTP
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.DDL.Headers
|
||||
import Hasura.RQL.Types
|
||||
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery as EL
|
||||
import qualified Hasura.GraphQL.Execute.Plan as EP
|
||||
import qualified Hasura.GraphQL.Execute.Query as EQ
|
||||
|
||||
import qualified Hasura.GraphQL.Resolve as GR
|
||||
import qualified Hasura.GraphQL.Validate as VQ
|
||||
import qualified Hasura.GraphQL.Validate.Types as VT
|
||||
|
||||
data GQExecPlan
|
||||
= GExPHasura !GCtx !VQ.RootSelSet
|
||||
-- The current execution plan of a graphql operation, it is
|
||||
-- currently, either local pg execution or a remote execution
|
||||
--
|
||||
-- The 'a' is parameterised so this AST can represent
|
||||
-- intermediate passes
|
||||
data GQExecPlan a
|
||||
= GExPHasura !a
|
||||
| GExPRemote !RemoteSchemaInfo !G.TypedOperationDefinition
|
||||
deriving (Functor, Foldable, Traversable)
|
||||
|
||||
getExecPlan
|
||||
-- Enforces the current limitation
|
||||
assertSameLocationNodes
|
||||
:: (MonadError QErr m) => [VT.TypeLoc] -> m VT.TypeLoc
|
||||
assertSameLocationNodes typeLocs =
|
||||
case Set.toList (Set.fromList typeLocs) of
|
||||
-- this shouldn't happen
|
||||
[] -> return VT.HasuraType
|
||||
[loc] -> return loc
|
||||
_ -> throw400 NotSupported msg
|
||||
where
|
||||
msg = "cannot mix top level fields from two different graphql servers"
|
||||
|
||||
-- TODO: we should fix this function asap
|
||||
-- as this will fail when there is a fragment at the top level
|
||||
getTopLevelNodes :: G.TypedOperationDefinition -> [G.Name]
|
||||
getTopLevelNodes opDef =
|
||||
mapMaybe f $ G._todSelectionSet opDef
|
||||
where
|
||||
f = \case
|
||||
G.SelectionField fld -> Just $ G._fName fld
|
||||
G.SelectionFragmentSpread _ -> Nothing
|
||||
G.SelectionInlineFragment _ -> Nothing
|
||||
|
||||
gatherTypeLocs :: GCtx -> [G.Name] -> [VT.TypeLoc]
|
||||
gatherTypeLocs gCtx nodes =
|
||||
catMaybes $ flip map nodes $ \node ->
|
||||
VT._fiLoc <$> Map.lookup node schemaNodes
|
||||
where
|
||||
schemaNodes =
|
||||
let qr = VT._otiFields $ _gQueryRoot gCtx
|
||||
mr = VT._otiFields <$> _gMutRoot gCtx
|
||||
in maybe qr (Map.union qr) mr
|
||||
|
||||
-- This is for when the graphql query is validated
|
||||
type ExecPlanPartial
|
||||
= GQExecPlan (GCtx, VQ.RootSelSet, [G.VariableDefinition])
|
||||
|
||||
getExecPlanPartial
|
||||
:: (MonadError QErr m)
|
||||
=> UserInfo
|
||||
-> SchemaCache
|
||||
-> GraphQLRequest
|
||||
-> m GQExecPlan
|
||||
getExecPlan userInfo sc req = do
|
||||
-> GQLReqParsed
|
||||
-> m ExecPlanPartial
|
||||
getExecPlanPartial userInfo sc req = do
|
||||
|
||||
(gCtx, _) <- flip runStateT sc $ getGCtx (userRole userInfo) gCtxRoleMap
|
||||
(gCtx, _) <- flip runStateT sc $ getGCtx (userRole userInfo) gCtxRoleMap
|
||||
queryParts <- flip runReaderT gCtx $ VQ.getQueryParts req
|
||||
|
||||
let opDef = VQ.qpOpDef queryParts
|
||||
@ -53,13 +116,195 @@ getExecPlan userInfo sc req = do
|
||||
typeLoc <- assertSameLocationNodes typeLocs
|
||||
|
||||
case typeLoc of
|
||||
VT.HasuraType ->
|
||||
GExPHasura gCtx <$> runReaderT (VQ.validateGQ queryParts) gCtx
|
||||
VT.HasuraType -> do
|
||||
rootSelSet <- runReaderT (VQ.validateGQ queryParts) gCtx
|
||||
let varDefs = G._todVariableDefinitions $ VQ.qpOpDef queryParts
|
||||
return $ GExPHasura (gCtx, rootSelSet, varDefs)
|
||||
VT.RemoteType _ rsi ->
|
||||
return $ GExPRemote rsi opDef
|
||||
where
|
||||
gCtxRoleMap = scGCtxMap sc
|
||||
|
||||
-- An execution operation, in case of
|
||||
-- queries and mutations it is just a transaction
|
||||
-- to be executed
|
||||
data ExecOp
|
||||
= ExOpQuery !LazyRespTx
|
||||
| ExOpMutation !LazyRespTx
|
||||
| ExOpSubs !EL.LiveQueryOp
|
||||
|
||||
-- The graphql query is resolved into an execution operation
|
||||
type ExecPlanResolved
|
||||
= GQExecPlan ExecOp
|
||||
|
||||
getResolvedExecPlan
|
||||
:: (MonadError QErr m, MonadIO m)
|
||||
=> PGExecCtx
|
||||
-> EP.PlanCache
|
||||
-> UserInfo
|
||||
-> SQLGenCtx
|
||||
-> SchemaCache
|
||||
-> SchemaCacheVer
|
||||
-> GQLReqUnparsed
|
||||
-> m ExecPlanResolved
|
||||
getResolvedExecPlan pgExecCtx planCache userInfo sqlGenCtx
|
||||
sc scVer reqUnparsed = do
|
||||
planM <- liftIO $ EP.getPlan scVer (userRole userInfo)
|
||||
opNameM queryStr planCache
|
||||
let usrVars = userVars userInfo
|
||||
case planM of
|
||||
-- plans are only for queries and subscriptions
|
||||
Just plan -> GExPHasura <$> case plan of
|
||||
EP.RPQuery queryPlan ->
|
||||
ExOpQuery <$> EQ.queryOpFromPlan usrVars queryVars queryPlan
|
||||
EP.RPSubs subsPlan ->
|
||||
ExOpSubs <$> EL.subsOpFromPlan pgExecCtx queryVars subsPlan
|
||||
Nothing -> noExistingPlan
|
||||
where
|
||||
GQLReq opNameM queryStr queryVars = reqUnparsed
|
||||
addPlanToCache plan =
|
||||
liftIO $ EP.addPlan scVer (userRole userInfo)
|
||||
opNameM queryStr plan planCache
|
||||
noExistingPlan = do
|
||||
req <- toParsed reqUnparsed
|
||||
partialExecPlan <- getExecPlanPartial userInfo sc req
|
||||
forM partialExecPlan $ \(gCtx, rootSelSet, varDefs) ->
|
||||
case rootSelSet of
|
||||
VQ.RMutation selSet ->
|
||||
ExOpMutation <$> getMutOp gCtx sqlGenCtx userInfo selSet
|
||||
VQ.RQuery selSet -> do
|
||||
(queryTx, planM) <- getQueryOp gCtx sqlGenCtx
|
||||
userInfo selSet varDefs
|
||||
mapM_ (addPlanToCache . EP.RPQuery) planM
|
||||
return $ ExOpQuery queryTx
|
||||
VQ.RSubscription fld -> do
|
||||
(lqOp, planM) <- getSubsOp pgExecCtx gCtx sqlGenCtx
|
||||
userInfo reqUnparsed varDefs fld
|
||||
mapM_ (addPlanToCache . EP.RPSubs) planM
|
||||
return $ ExOpSubs lqOp
|
||||
|
||||
-- Monad for resolving a hasura query/mutation
|
||||
type E m =
|
||||
ReaderT ( UserInfo
|
||||
, OpCtxMap
|
||||
, TypeMap
|
||||
, FieldMap
|
||||
, OrdByCtx
|
||||
, InsCtxMap
|
||||
, SQLGenCtx
|
||||
) (ExceptT QErr m)
|
||||
|
||||
runE
|
||||
:: (MonadError QErr m)
|
||||
=> GCtx
|
||||
-> SQLGenCtx
|
||||
-> UserInfo
|
||||
-> E m a
|
||||
-> m a
|
||||
runE ctx sqlGenCtx userInfo action = do
|
||||
res <- runExceptT $ runReaderT action
|
||||
(userInfo, opCtxMap, typeMap, fldMap, ordByCtx, insCtxMap, sqlGenCtx)
|
||||
either throwError return res
|
||||
where
|
||||
opCtxMap = _gOpCtxMap ctx
|
||||
typeMap = _gTypes ctx
|
||||
fldMap = _gFields ctx
|
||||
ordByCtx = _gOrdByCtx ctx
|
||||
insCtxMap = _gInsCtxMap ctx
|
||||
|
||||
getQueryOp
|
||||
:: (MonadError QErr m)
|
||||
=> GCtx
|
||||
-> SQLGenCtx
|
||||
-> UserInfo
|
||||
-> VQ.SelSet
|
||||
-> [G.VariableDefinition]
|
||||
-> m (LazyRespTx, Maybe EQ.ReusableQueryPlan)
|
||||
getQueryOp gCtx sqlGenCtx userInfo fields varDefs =
|
||||
runE gCtx sqlGenCtx userInfo $ EQ.convertQuerySelSet varDefs fields
|
||||
|
||||
mutationRootName :: Text
|
||||
mutationRootName = "mutation_root"
|
||||
|
||||
resolveMutSelSet
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has UserInfo r
|
||||
, Has OpCtxMap r
|
||||
, Has FieldMap r
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
, Has InsCtxMap r
|
||||
)
|
||||
=> VQ.SelSet
|
||||
-> m LazyRespTx
|
||||
resolveMutSelSet fields = do
|
||||
aliasedTxs <- forM (toList fields) $ \fld -> do
|
||||
fldRespTx <- case VQ._fName fld of
|
||||
"__typename" -> return $ return $ encJFromJValue mutationRootName
|
||||
_ -> liftTx <$> GR.mutFldToTx fld
|
||||
return (G.unName $ G.unAlias $ VQ._fAlias fld, fldRespTx)
|
||||
|
||||
-- combines all transactions into a single transaction
|
||||
return $ toSingleTx aliasedTxs
|
||||
where
|
||||
-- A list of aliased transactions for eg
|
||||
-- [("f1", Tx r1), ("f2", Tx r2)]
|
||||
-- are converted into a single transaction as follows
|
||||
-- Tx {"f1": r1, "f2": r2}
|
||||
toSingleTx :: [(Text, LazyRespTx)] -> LazyRespTx
|
||||
toSingleTx aliasedTxs =
|
||||
fmap encJFromAssocList $
|
||||
forM aliasedTxs $ \(al, tx) -> (,) al <$> tx
|
||||
|
||||
getMutOp
|
||||
:: (MonadError QErr m)
|
||||
=> GCtx
|
||||
-> SQLGenCtx
|
||||
-> UserInfo
|
||||
-> VQ.SelSet
|
||||
-> m LazyRespTx
|
||||
getMutOp ctx sqlGenCtx userInfo selSet =
|
||||
runE ctx sqlGenCtx userInfo $ resolveMutSelSet selSet
|
||||
|
||||
getSubsOpM
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has OpCtxMap r
|
||||
, Has FieldMap r
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
, Has UserInfo r
|
||||
, MonadIO m
|
||||
)
|
||||
=> PGExecCtx
|
||||
-> GQLReqUnparsed
|
||||
-> [G.VariableDefinition]
|
||||
-> VQ.Field
|
||||
-> m (EL.LiveQueryOp, Maybe EL.SubsPlan)
|
||||
getSubsOpM pgExecCtx req varDefs fld =
|
||||
case VQ._fName fld of
|
||||
"__typename" ->
|
||||
throwVE "you cannot create a subscription on '__typename' field"
|
||||
_ -> do
|
||||
astUnresolved <- GR.queryFldToPGAST fld
|
||||
EL.subsOpFromPGAST pgExecCtx req varDefs (VQ._fAlias fld, astUnresolved)
|
||||
|
||||
getSubsOp
|
||||
:: ( MonadError QErr m
|
||||
, MonadIO m
|
||||
)
|
||||
=> PGExecCtx
|
||||
-> GCtx
|
||||
-> SQLGenCtx
|
||||
-> UserInfo
|
||||
-> GQLReqUnparsed
|
||||
-> [G.VariableDefinition]
|
||||
-> VQ.Field
|
||||
-> m (EL.LiveQueryOp, Maybe EL.SubsPlan)
|
||||
getSubsOp pgExecCtx gCtx sqlGenCtx userInfo req varDefs fld =
|
||||
runE gCtx sqlGenCtx userInfo $ getSubsOpM pgExecCtx req varDefs fld
|
||||
|
||||
execRemoteGQ
|
||||
:: (MonadIO m, MonadError QErr m)
|
||||
=> HTTP.Manager
|
||||
@ -96,35 +341,3 @@ execRemoteGQ manager userInfo reqHdrs q rsi opDef = do
|
||||
, "Accept-Language", "Accept-Datetime"
|
||||
, "Cache-Control", "Connection", "DNT"
|
||||
]
|
||||
|
||||
assertSameLocationNodes
|
||||
:: (MonadError QErr m) => [VT.TypeLoc] -> m VT.TypeLoc
|
||||
assertSameLocationNodes typeLocs =
|
||||
case Set.toList (Set.fromList typeLocs) of
|
||||
-- this shouldn't happen
|
||||
[] -> return VT.HasuraType
|
||||
[loc] -> return loc
|
||||
_ -> throw400 NotSupported msg
|
||||
where
|
||||
msg = "cannot mix top level fields from two different graphql servers"
|
||||
|
||||
-- TODO: we should fix this function asap
|
||||
-- as this will fail when there is a fragment at the top level
|
||||
getTopLevelNodes :: G.TypedOperationDefinition -> [G.Name]
|
||||
getTopLevelNodes opDef =
|
||||
mapMaybe f $ G._todSelectionSet opDef
|
||||
where
|
||||
f = \case
|
||||
G.SelectionField fld -> Just $ G._fName fld
|
||||
G.SelectionFragmentSpread _ -> Nothing
|
||||
G.SelectionInlineFragment _ -> Nothing
|
||||
|
||||
gatherTypeLocs :: GCtx -> [G.Name] -> [VT.TypeLoc]
|
||||
gatherTypeLocs gCtx nodes =
|
||||
catMaybes $ flip map nodes $ \node ->
|
||||
VT._fiLoc <$> Map.lookup node schemaNodes
|
||||
where
|
||||
schemaNodes =
|
||||
let qr = VT._otiFields $ _gQueryRoot gCtx
|
||||
mr = VT._otiFields <$> _gMutRoot gCtx
|
||||
in maybe qr (Map.union qr) mr
|
||||
|
283
server/src-lib/Hasura/GraphQL/Execute/LiveQuery.hs
Normal file
283
server/src-lib/Hasura/GraphQL/Execute/LiveQuery.hs
Normal file
@ -0,0 +1,283 @@
|
||||
module Hasura.GraphQL.Execute.LiveQuery
|
||||
( RefetchInterval
|
||||
, refetchIntervalFromMilli
|
||||
, LQM.BatchSize
|
||||
, LQM.mkBatchSize
|
||||
, LQM.MxOpts
|
||||
, LQM.mkMxOpts
|
||||
, LQF.FallbackOpts
|
||||
, LQF.mkFallbackOpts
|
||||
, LQOpts
|
||||
, mkLQOpts
|
||||
|
||||
, LiveQueriesState
|
||||
, initLiveQueriesState
|
||||
, dumpLiveQueriesState
|
||||
|
||||
, LiveQueryOp
|
||||
, LiveQueryId
|
||||
, addLiveQuery
|
||||
, removeLiveQuery
|
||||
|
||||
, SubsPlan
|
||||
, subsOpFromPlan
|
||||
, subsOpFromPGAST
|
||||
) where
|
||||
|
||||
import Data.Has
|
||||
|
||||
import qualified Control.Concurrent.STM as STM
|
||||
import qualified Data.Aeson as J
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Data.HashSet as Set
|
||||
import qualified Data.Text as T
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Database.PG.Query.Connection as Q
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery.Fallback as LQF
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery.Multiplexed as LQM
|
||||
import qualified Hasura.GraphQL.Resolve as GR
|
||||
import qualified Hasura.GraphQL.Transport.HTTP.Protocol as GH
|
||||
import qualified Hasura.GraphQL.Validate as GV
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
import Hasura.Db
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Execute.LiveQuery.Types
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.DML.Select (asSingleRowJsonResp)
|
||||
import Hasura.RQL.Types
|
||||
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
data LQOpts
|
||||
= LQOpts
|
||||
{ _loMxOpts :: LQM.MxOpts
|
||||
, _loFallbackOpts :: LQF.FallbackOpts
|
||||
} deriving (Show, Eq)
|
||||
|
||||
mkLQOpts :: LQM.MxOpts -> LQF.FallbackOpts -> LQOpts
|
||||
mkLQOpts = LQOpts
|
||||
|
||||
data LiveQueriesState
|
||||
= LiveQueriesState
|
||||
{ _lqsMultiplexed :: !LQM.LiveQueriesState
|
||||
, _lqsFallback :: !LQF.LiveQueriesState
|
||||
, _lqsPGExecTx :: !PGExecCtx
|
||||
}
|
||||
|
||||
dumpLiveQueriesState
|
||||
:: LiveQueriesState -> IO J.Value
|
||||
dumpLiveQueriesState (LiveQueriesState mx fallback _) = do
|
||||
mxJ <- LQM.dumpLiveQueriesState mx
|
||||
fallbackJ <- LQF.dumpLiveQueriesState fallback
|
||||
return $ J.object
|
||||
[ "fallback" J..= fallbackJ
|
||||
, "multiplexed" J..= mxJ
|
||||
]
|
||||
|
||||
initLiveQueriesState
|
||||
:: LQOpts
|
||||
-> PGExecCtx
|
||||
-> IO LiveQueriesState
|
||||
initLiveQueriesState (LQOpts mxOpts fallbackOpts) pgExecCtx = do
|
||||
(mxMap, fallbackMap) <- STM.atomically $
|
||||
(,) <$> LQM.initLiveQueriesState mxOpts
|
||||
<*> LQF.initLiveQueriesState fallbackOpts
|
||||
return $ LiveQueriesState mxMap fallbackMap pgExecCtx
|
||||
|
||||
data LiveQueryOp
|
||||
= LQMultiplexed !LQM.MxOp
|
||||
| LQFallback !LQF.FallbackOp
|
||||
|
||||
data LiveQueryId
|
||||
= LQIMultiplexed !LQM.LiveQueryId
|
||||
| LQIFallback !LQF.LiveQueryId
|
||||
|
||||
addLiveQuery
|
||||
:: LiveQueriesState
|
||||
-> LiveQueryOp
|
||||
-- the action to be executed when result changes
|
||||
-> OnChange
|
||||
-> IO LiveQueryId
|
||||
addLiveQuery lqState liveQOp onResultAction =
|
||||
case liveQOp of
|
||||
LQMultiplexed mxOp ->
|
||||
LQIMultiplexed <$> LQM.addLiveQuery pgExecCtx mxMap mxOp onResultAction
|
||||
LQFallback fallbackOp ->
|
||||
LQIFallback <$> LQF.addLiveQuery
|
||||
pgExecCtx fallbackMap fallbackOp onResultAction
|
||||
where
|
||||
LiveQueriesState mxMap fallbackMap pgExecCtx = lqState
|
||||
|
||||
removeLiveQuery
|
||||
:: LiveQueriesState
|
||||
-- the query and the associated operation
|
||||
-> LiveQueryId
|
||||
-> IO ()
|
||||
removeLiveQuery lqState = \case
|
||||
LQIMultiplexed lqId -> LQM.removeLiveQuery mxMap lqId
|
||||
LQIFallback lqId -> LQF.removeLiveQuery fallbackMap lqId
|
||||
where
|
||||
LiveQueriesState mxMap fallbackMap _ = lqState
|
||||
|
||||
data SubsPlan
|
||||
= SubsPlan
|
||||
{ _sfMxOpCtx :: !LQM.MxOpCtx
|
||||
, _sfVariableTypes :: !GV.VarPGTypes
|
||||
}
|
||||
|
||||
instance J.ToJSON SubsPlan where
|
||||
toJSON (SubsPlan opCtx varTypes) =
|
||||
J.object [ "mx_op_ctx" J..= opCtx
|
||||
, "variable_types" J..= varTypes
|
||||
]
|
||||
|
||||
collectNonNullableVars
|
||||
:: (MonadState GV.VarPGTypes m)
|
||||
=> GR.UnresolvedVal -> m GR.UnresolvedVal
|
||||
collectNonNullableVars val = do
|
||||
case val of
|
||||
GR.UVPG annPGVal -> do
|
||||
let GR.AnnPGVal varM isNullable colTy _ = annPGVal
|
||||
case (varM, isNullable) of
|
||||
(Just var, False) -> modify (Map.insert var colTy)
|
||||
_ -> return ()
|
||||
_ -> return ()
|
||||
return val
|
||||
|
||||
type TextEncodedVariables
|
||||
= Map.HashMap G.Variable TxtEncodedPGVal
|
||||
|
||||
toMultiplexedQueryVar
|
||||
:: (MonadState GV.AnnPGVarVals m)
|
||||
=> GR.UnresolvedVal -> m S.SQLExp
|
||||
toMultiplexedQueryVar = \case
|
||||
GR.UVPG annPGVal ->
|
||||
let GR.AnnPGVal varM isNullable colTy colVal = annPGVal
|
||||
in case (varM, isNullable) of
|
||||
-- we don't check for nullability as
|
||||
-- this is only used for reusable plans
|
||||
-- the check has to be made before this
|
||||
(Just var, _) -> do
|
||||
modify $ Map.insert var (colTy, colVal)
|
||||
return $ fromResVars colTy
|
||||
[ "variables"
|
||||
, G.unName $ G.unVariable var
|
||||
]
|
||||
_ -> return $ toTxtValue colTy colVal
|
||||
-- TODO: check the logic around colTy and session variable's type
|
||||
GR.UVSessVar colTy sessVar ->
|
||||
return $ fromResVars colTy [ "user", T.toLower sessVar]
|
||||
GR.UVSQL sqlExp -> return sqlExp
|
||||
where
|
||||
fromResVars colTy jPath =
|
||||
S.withTyAnn colTy $ S.SEOpApp (S.SQLOp "#>>")
|
||||
[ S.SEQIden $ S.QIden (S.QualIden $ Iden "_subs")
|
||||
(Iden "result_vars")
|
||||
, S.SEArray $ map S.SELit jPath
|
||||
]
|
||||
|
||||
subsOpFromPGAST
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has UserInfo r
|
||||
, MonadIO m
|
||||
)
|
||||
=> PGExecCtx
|
||||
-> GH.GQLReqUnparsed
|
||||
-> [G.VariableDefinition]
|
||||
-> (G.Alias, GR.QueryRootFldUnresolved)
|
||||
-> m (LiveQueryOp, Maybe SubsPlan)
|
||||
subsOpFromPGAST pgExecCtx reqUnparsed varDefs (fldAls, astUnresolved) = do
|
||||
userInfo <- asks getter
|
||||
(_, varTypes) <- flip runStateT mempty $ GR.traverseQueryRootFldAST
|
||||
collectNonNullableVars astUnresolved
|
||||
-- can the subscription be multiplexed?
|
||||
if Set.fromList (Map.keys varTypes) == allVars
|
||||
then mkMultiplexedOp userInfo varTypes
|
||||
else mkFallbackOp userInfo
|
||||
where
|
||||
allVars = Set.fromList $ map G._vdVariable varDefs
|
||||
|
||||
-- multiplexed subscription
|
||||
mkMultiplexedOp userInfo varTypes = do
|
||||
(astResolved, annVarVals) <-
|
||||
flip runStateT mempty $ GR.traverseQueryRootFldAST
|
||||
toMultiplexedQueryVar astUnresolved
|
||||
let mxOpCtx = LQM.mkMxOpCtx userInfo reqUnparsed fldAls $
|
||||
GR.toPGQuery astResolved
|
||||
txtEncodedVars <- validateAnnVarValsOnPg pgExecCtx annVarVals
|
||||
let mxOp = (mxOpCtx, txtEncodedVars)
|
||||
return (LQMultiplexed mxOp, Just $ SubsPlan mxOpCtx varTypes)
|
||||
|
||||
-- fallback tx subscription
|
||||
mkFallbackOp userInfo = do
|
||||
(astResolved, prepArgs) <-
|
||||
flip runStateT mempty $ GR.traverseQueryRootFldAST
|
||||
GR.resolveValPrep astUnresolved
|
||||
let tx = withUserInfo userInfo $ liftTx $
|
||||
asSingleRowJsonResp (GR.toPGQuery astResolved) $ toList prepArgs
|
||||
fallbackOp = LQF.mkFallbackOp userInfo reqUnparsed $ withAlias tx
|
||||
return (LQFallback fallbackOp, Nothing)
|
||||
|
||||
fldAlsT = G.unName $ G.unAlias fldAls
|
||||
withAlias tx =
|
||||
encJFromAssocList . pure . (,) fldAlsT <$> tx
|
||||
|
||||
validateAnnVarValsOnPg
|
||||
:: ( MonadError QErr m
|
||||
, MonadIO m
|
||||
)
|
||||
=> PGExecCtx
|
||||
-> GV.AnnPGVarVals
|
||||
-> m TextEncodedVariables
|
||||
validateAnnVarValsOnPg pgExecCtx annVarVals = do
|
||||
let valSel = mkValidationSel $ Map.elems annVarVals
|
||||
|
||||
Q.Discard _ <- runTx' $ liftTx $
|
||||
Q.rawQE valPgErrHandler (Q.fromBuilder $ toSQL valSel) [] False
|
||||
return $ fmap (txtEncodedPGVal . snd) annVarVals
|
||||
|
||||
where
|
||||
mkExtrs = map (flip S.Extractor Nothing . uncurry toTxtValue)
|
||||
mkValidationSel vars =
|
||||
S.mkSelect { S.selExtr = mkExtrs vars }
|
||||
runTx' tx = do
|
||||
res <- liftIO $ runExceptT (runLazyTx' pgExecCtx tx)
|
||||
liftEither res
|
||||
|
||||
valPgErrHandler :: Q.PGTxErr -> QErr
|
||||
valPgErrHandler txErr =
|
||||
fromMaybe (defaultTxErrorHandler txErr) $ do
|
||||
stmtErr <- Q.getPGStmtErr txErr
|
||||
codeMsg <- getPGCodeMsg stmtErr
|
||||
(qErrCode, qErrMsg) <- extractError codeMsg
|
||||
return $ err400 qErrCode qErrMsg
|
||||
where
|
||||
getPGCodeMsg pged =
|
||||
(,) <$> Q.edStatusCode pged <*> Q.edMessage pged
|
||||
extractError = \case
|
||||
-- invalid text representation
|
||||
("22P02", msg) -> return (DataException, msg)
|
||||
-- invalid parameter value
|
||||
("22023", msg) -> return (DataException, msg)
|
||||
-- invalid input values
|
||||
("22007", msg) -> return (DataException, msg)
|
||||
_ -> Nothing
|
||||
|
||||
-- use the existing plan and new variables to create a pg query
|
||||
subsOpFromPlan
|
||||
:: ( MonadError QErr m
|
||||
, MonadIO m
|
||||
)
|
||||
=> PGExecCtx
|
||||
-> Maybe GH.VariableValues
|
||||
-> SubsPlan
|
||||
-> m LiveQueryOp
|
||||
subsOpFromPlan pgExecCtx varValsM (SubsPlan mxOpCtx varTypes) = do
|
||||
annVarVals <- GV.getAnnPGVarVals varTypes varValsM
|
||||
txtEncodedVars <- validateAnnVarValsOnPg pgExecCtx annVarVals
|
||||
return $ LQMultiplexed (mxOpCtx, txtEncodedVars)
|
249
server/src-lib/Hasura/GraphQL/Execute/LiveQuery/Fallback.hs
Normal file
249
server/src-lib/Hasura/GraphQL/Execute/LiveQuery/Fallback.hs
Normal file
@ -0,0 +1,249 @@
|
||||
module Hasura.GraphQL.Execute.LiveQuery.Fallback
|
||||
( RefetchInterval
|
||||
, refetchIntervalFromMilli
|
||||
, FallbackOpts
|
||||
, mkFallbackOpts
|
||||
|
||||
, LiveQueriesState
|
||||
, initLiveQueriesState
|
||||
, dumpLiveQueriesState
|
||||
|
||||
, FallbackOp
|
||||
, mkFallbackOp
|
||||
, LiveQueryId
|
||||
, addLiveQuery
|
||||
, removeLiveQuery
|
||||
) where
|
||||
|
||||
import qualified Control.Concurrent.Async as A
|
||||
import qualified Control.Concurrent.STM as STM
|
||||
import qualified Data.Aeson as J
|
||||
import qualified ListT
|
||||
import qualified StmContainers.Map as STMMap
|
||||
|
||||
import Control.Concurrent (threadDelay)
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Execute.LiveQuery.Types
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types
|
||||
|
||||
data LiveQuery
|
||||
= LiveQuery
|
||||
{ _lqUser :: !UserInfo
|
||||
, _lqRequest :: !GQLReqUnparsed
|
||||
} deriving (Show, Eq, Generic)
|
||||
|
||||
instance J.ToJSON LiveQuery where
|
||||
toJSON (LiveQuery user req) =
|
||||
J.object [ "user" J..= userVars user
|
||||
, "request" J..= req
|
||||
]
|
||||
|
||||
instance Hashable LiveQuery
|
||||
|
||||
data LQHandler
|
||||
= LQHandler
|
||||
-- the tx to be executed
|
||||
{ _lqhRespTx :: !LazyRespTx
|
||||
-- previous result
|
||||
, _lqhPrevRes :: !RespTV
|
||||
-- the actions that have been run previously
|
||||
-- we run these if the response changes
|
||||
, _lqhCurOps :: !Sinks
|
||||
-- we run these operations regardless
|
||||
-- and then merge them with current operations
|
||||
, _lqhNewOps :: !Sinks
|
||||
}
|
||||
|
||||
data FallbackOpts
|
||||
= FallbackOpts
|
||||
{ _foRefetchInterval :: !RefetchInterval
|
||||
} deriving (Show, Eq)
|
||||
|
||||
instance J.ToJSON FallbackOpts where
|
||||
toJSON (FallbackOpts refetchInterval) =
|
||||
J.object [ "refetch_delay" J..= refetchInterval
|
||||
]
|
||||
|
||||
-- 1 second
|
||||
defaultRefetchInterval :: RefetchInterval
|
||||
defaultRefetchInterval =
|
||||
refetchIntervalFromMilli 1000
|
||||
|
||||
mkFallbackOpts
|
||||
:: Maybe RefetchInterval
|
||||
-> FallbackOpts
|
||||
mkFallbackOpts refetchIntervalM =
|
||||
FallbackOpts
|
||||
(fromMaybe defaultRefetchInterval refetchIntervalM)
|
||||
|
||||
data LiveQueriesState
|
||||
= LiveQueriesState
|
||||
{ _lqsOptions :: !FallbackOpts
|
||||
, _lqsLiveQueryMap :: !LiveQueryMap
|
||||
}
|
||||
|
||||
dumpLiveQueriesState :: LiveQueriesState -> IO J.Value
|
||||
dumpLiveQueriesState (LiveQueriesState opts lqMap) = do
|
||||
lqMapJ <- dumpLiveQueryMap lqMap
|
||||
return $ J.object
|
||||
[ "options" J..= opts
|
||||
, "live_queries_map" J..= lqMapJ
|
||||
]
|
||||
|
||||
initLiveQueriesState
|
||||
:: FallbackOpts
|
||||
-> STM.STM LiveQueriesState
|
||||
initLiveQueriesState lqOptions =
|
||||
LiveQueriesState
|
||||
lqOptions
|
||||
<$> STMMap.new
|
||||
|
||||
data LiveQueryId
|
||||
= LiveQueryId
|
||||
{ _lqiQuery :: !LiveQuery
|
||||
, _lqiSink :: !SinkId
|
||||
}
|
||||
|
||||
type LiveQueryMap = STMMap.Map LiveQuery (LQHandler, ThreadTM)
|
||||
|
||||
dumpLiveQueryMap :: LiveQueryMap -> IO J.Value
|
||||
dumpLiveQueryMap lqMap =
|
||||
fmap J.toJSON $ STM.atomically $ do
|
||||
entries <- ListT.toList $ STMMap.listT lqMap
|
||||
forM entries $ \(lq, (lqHandler, threadRef)) -> do
|
||||
prevResHash <- STM.readTVar $ _lqhPrevRes lqHandler
|
||||
threadId <- A.asyncThreadId <$> STM.readTMVar threadRef
|
||||
curOps <- toListTMap $ _lqhCurOps lqHandler
|
||||
newOps <- toListTMap $ _lqhNewOps lqHandler
|
||||
return $ J.object
|
||||
[ "query" J..= lq
|
||||
, "thread_id" J..= show threadId
|
||||
, "current_ops" J..= map fst curOps
|
||||
, "new_ops" J..= map fst newOps
|
||||
, "previous_result_hash" J..= prevResHash
|
||||
]
|
||||
|
||||
removeLiveQuery
|
||||
:: LiveQueriesState
|
||||
-- the query and the associated operation
|
||||
-> LiveQueryId
|
||||
-> IO ()
|
||||
removeLiveQuery lqState (LiveQueryId liveQ k) = do
|
||||
|
||||
-- clean the handler's state
|
||||
threadRefM <- STM.atomically $ do
|
||||
lqHandlerM <- STMMap.lookup liveQ lqMap
|
||||
maybe (return Nothing) cleanLQHandler lqHandlerM
|
||||
|
||||
-- cancel the polling thread
|
||||
onJust threadRefM A.cancel
|
||||
|
||||
where
|
||||
lqMap = _lqsLiveQueryMap lqState
|
||||
cleanLQHandler (handler, threadRef) = do
|
||||
let curOps = _lqhCurOps handler
|
||||
newOps = _lqhNewOps handler
|
||||
deleteTMap k curOps
|
||||
deleteTMap k newOps
|
||||
cancelPollThread <- (&&)
|
||||
<$> nullTMap curOps
|
||||
<*> nullTMap newOps
|
||||
-- if this happens to be the last operation, take the
|
||||
-- ref for the polling thread to cancel it
|
||||
if cancelPollThread then do
|
||||
STMMap.delete liveQ lqMap
|
||||
Just <$> STM.takeTMVar threadRef
|
||||
else return Nothing
|
||||
|
||||
-- the transaction associated with this query
|
||||
type FallbackOp = (LiveQuery, LazyRespTx)
|
||||
|
||||
mkFallbackOp
|
||||
:: UserInfo -> GQLReqUnparsed
|
||||
-> LazyRespTx -> FallbackOp
|
||||
mkFallbackOp userInfo req tx =
|
||||
(LiveQuery userInfo req, tx)
|
||||
|
||||
|
||||
addLiveQuery
|
||||
:: PGExecCtx
|
||||
-> LiveQueriesState
|
||||
-- the query
|
||||
-> FallbackOp
|
||||
-- the action to be executed when result changes
|
||||
-> OnChange
|
||||
-> IO LiveQueryId
|
||||
addLiveQuery pgExecCtx lqState (liveQ, respTx) onResultAction= do
|
||||
|
||||
sinkId <- newSinkId
|
||||
|
||||
-- a handler is returned only when it is newly created
|
||||
handlerM <- STM.atomically $ do
|
||||
lqHandlerM <- STMMap.lookup liveQ lqMap
|
||||
maybe (newHandler sinkId) (addToExistingHandler sinkId) lqHandlerM
|
||||
|
||||
-- we can then attach a polling thread if it is new
|
||||
-- the livequery can only be cancelled after putTMVar
|
||||
onJust handlerM $ \(handler, pollerThreadTM) -> do
|
||||
threadRef <- A.async $ forever $ do
|
||||
pollQuery pgExecCtx handler
|
||||
threadDelay $ refetchIntervalToMicro refetchInterval
|
||||
STM.atomically $ STM.putTMVar pollerThreadTM threadRef
|
||||
|
||||
return $ LiveQueryId liveQ sinkId
|
||||
|
||||
where
|
||||
|
||||
LiveQueriesState lqOpts lqMap = lqState
|
||||
FallbackOpts refetchInterval = lqOpts
|
||||
|
||||
addToExistingHandler sinkId (handler, _) = do
|
||||
insertTMap onResultAction sinkId $ _lqhNewOps handler
|
||||
return Nothing
|
||||
|
||||
newHandler sinkId = do
|
||||
handler <- LQHandler
|
||||
<$> return respTx
|
||||
<*> STM.newTVar Nothing
|
||||
<*> newTMap
|
||||
<*> newTMap
|
||||
insertTMap onResultAction sinkId $ _lqhNewOps handler
|
||||
asyncRefTM <- STM.newEmptyTMVar
|
||||
STMMap.insert (handler, asyncRefTM) liveQ lqMap
|
||||
return $ Just (handler, asyncRefTM)
|
||||
|
||||
pollQuery
|
||||
:: PGExecCtx
|
||||
-> LQHandler
|
||||
-> IO ()
|
||||
pollQuery pgExecCtx (LQHandler respTx respTV curOpsTV newOpsTV) = do
|
||||
|
||||
resOrErr <- runExceptT $ runLazyTx pgExecCtx respTx
|
||||
|
||||
let (resp, respHashM) = case encJToLBS <$> resOrErr of
|
||||
Left e -> (GQExecError [encodeGQErr False e], Nothing)
|
||||
Right lbs -> (GQSuccess lbs, Just $ mkRespHash lbs)
|
||||
|
||||
-- extract the current and new operations
|
||||
(curOps, newOps) <- STM.atomically $ do
|
||||
curOpsL <- toListTMap curOpsTV
|
||||
newOpsL <- toListTMap newOpsTV
|
||||
forM_ newOpsL $ \(k, action) -> insertTMap action k curOpsTV
|
||||
resetTMap newOpsTV
|
||||
return (curOpsL, newOpsL)
|
||||
|
||||
runOperations resp newOps
|
||||
|
||||
-- write to the current websockets if needed
|
||||
prevRespHashM <- STM.readTVarIO respTV
|
||||
when (isExecError resp || respHashM /= prevRespHashM) $ do
|
||||
runOperations resp curOps
|
||||
STM.atomically $ STM.writeTVar respTV respHashM
|
||||
|
||||
where
|
||||
runOperation resp action = action resp
|
||||
runOperations resp =
|
||||
void . A.mapConcurrently (runOperation resp . snd)
|
568
server/src-lib/Hasura/GraphQL/Execute/LiveQuery/Multiplexed.hs
Normal file
568
server/src-lib/Hasura/GraphQL/Execute/LiveQuery/Multiplexed.hs
Normal file
@ -0,0 +1,568 @@
|
||||
module Hasura.GraphQL.Execute.LiveQuery.Multiplexed
|
||||
( BatchSize
|
||||
, mkBatchSize
|
||||
, RefetchInterval
|
||||
, refetchIntervalFromMilli
|
||||
, MxOpts
|
||||
, mkMxOpts
|
||||
|
||||
, LiveQueriesState
|
||||
, initLiveQueriesState
|
||||
, dumpLiveQueriesState
|
||||
|
||||
, MxOpCtx
|
||||
, mkMxOpCtx
|
||||
, MxOp
|
||||
|
||||
, LiveQueryId
|
||||
, addLiveQuery
|
||||
, removeLiveQuery
|
||||
) where
|
||||
|
||||
import Data.List (unfoldr)
|
||||
import Data.Word (Word32)
|
||||
import qualified ListT
|
||||
|
||||
import qualified Control.Concurrent.Async as A
|
||||
import qualified Control.Concurrent.STM as STM
|
||||
import qualified Data.Aeson.Extended as J
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Data.Time.Clock as Clock
|
||||
import qualified Data.UUID as UUID
|
||||
import qualified Data.UUID.V4 as UUID
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
import qualified StmContainers.Map as STMMap
|
||||
import qualified System.Metrics.Distribution as Metrics
|
||||
|
||||
import Control.Concurrent (threadDelay)
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Execute.LiveQuery.Types
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
-- remove these when array encoding is merged
|
||||
import qualified Database.PG.Query.PTI as PTI
|
||||
import qualified PostgreSQL.Binary.Encoding as PE
|
||||
|
||||
newtype RespId
|
||||
= RespId { unRespId :: UUID.UUID }
|
||||
deriving (Show, Eq, Hashable, Q.FromCol)
|
||||
|
||||
newtype RespIdList
|
||||
= RespIdList { unRespIdList :: [RespId] }
|
||||
deriving (Show, Eq)
|
||||
|
||||
instance Q.ToPrepArg RespIdList where
|
||||
toPrepVal (RespIdList l) =
|
||||
Q.toPrepValHelper PTI.unknown encoder $ map unRespId l
|
||||
where
|
||||
encoder =
|
||||
PE.array 2950 . PE.dimensionArray foldl'
|
||||
(PE.encodingArray . PE.uuid)
|
||||
|
||||
newRespId :: IO RespId
|
||||
newRespId = RespId <$> UUID.nextRandom
|
||||
|
||||
data LQGroup
|
||||
-- we don't need operation name here as a subscription will
|
||||
-- only have a single top level field
|
||||
= LQGroup
|
||||
{ _lgRole :: !RoleName
|
||||
, _lgGQLQueryText :: !GQLQueryText
|
||||
} deriving (Show, Eq, Generic)
|
||||
|
||||
instance Hashable LQGroup
|
||||
|
||||
instance J.ToJSON LQGroup where
|
||||
toJSON (LQGroup role query) =
|
||||
J.object [ "role" J..= role
|
||||
, "query" J..= query
|
||||
]
|
||||
|
||||
data MxOpts
|
||||
= MxOpts
|
||||
{ _moBatchSize :: !BatchSize
|
||||
, _moRefetchInterval :: !RefetchInterval
|
||||
} deriving (Show, Eq)
|
||||
|
||||
instance J.ToJSON MxOpts where
|
||||
toJSON (MxOpts batchSize refetchInterval) =
|
||||
J.object [ "batch_size" J..= batchSize
|
||||
, "refetch_delay" J..= refetchInterval
|
||||
]
|
||||
|
||||
-- 1 second
|
||||
defaultRefetchInterval :: RefetchInterval
|
||||
defaultRefetchInterval =
|
||||
refetchIntervalFromMilli 1000
|
||||
|
||||
mkMxOpts
|
||||
:: Maybe BatchSize
|
||||
-> Maybe RefetchInterval
|
||||
-> MxOpts
|
||||
mkMxOpts batchSizeM refetchIntervalM =
|
||||
MxOpts
|
||||
(fromMaybe defaultBatchSize batchSizeM)
|
||||
(fromMaybe defaultRefetchInterval refetchIntervalM)
|
||||
|
||||
data LiveQueriesState
|
||||
= LiveQueriesState
|
||||
{ _lqsOptions :: !MxOpts
|
||||
, _lqsLiveQueryMap :: !LiveQueryMap
|
||||
}
|
||||
|
||||
data RefetchMetrics
|
||||
= RefetchMetrics
|
||||
{ _rmSnapshot :: !Metrics.Distribution
|
||||
, _rmPush :: !Metrics.Distribution
|
||||
, _rmQuery :: !Metrics.Distribution
|
||||
, _rmTotal :: !Metrics.Distribution
|
||||
}
|
||||
|
||||
initRefetchMetrics :: IO RefetchMetrics
|
||||
initRefetchMetrics =
|
||||
RefetchMetrics
|
||||
<$> Metrics.new
|
||||
<*> Metrics.new
|
||||
<*> Metrics.new
|
||||
<*> Metrics.new
|
||||
|
||||
data ThreadState
|
||||
= ThreadState
|
||||
{ _tsThread :: !(A.Async ())
|
||||
, _tsMetrics :: !RefetchMetrics
|
||||
}
|
||||
|
||||
type LiveQueryMap
|
||||
= STMMap.Map LQGroup (LQHandler, STM.TMVar ThreadState)
|
||||
|
||||
initLiveQueriesState
|
||||
:: MxOpts
|
||||
-> STM.STM LiveQueriesState
|
||||
initLiveQueriesState lqOptions =
|
||||
LiveQueriesState
|
||||
lqOptions
|
||||
<$> STMMap.new
|
||||
|
||||
dumpLiveQueriesState :: LiveQueriesState -> IO J.Value
|
||||
dumpLiveQueriesState (LiveQueriesState opts lqMap) = do
|
||||
lqMapJ <- dumpLiveQueryMap lqMap
|
||||
return $ J.object
|
||||
[ "options" J..= opts
|
||||
, "live_queries_map" J..= lqMapJ
|
||||
]
|
||||
|
||||
dumpLiveQueryMap :: LiveQueryMap -> IO J.Value
|
||||
dumpLiveQueryMap lqMap =
|
||||
fmap J.toJSON $ do
|
||||
entries <- STM.atomically $ ListT.toList $ STMMap.listT lqMap
|
||||
forM entries $ \(lq, (lqHandler, threadRef)) -> do
|
||||
ThreadState threadId metrics <-
|
||||
STM.atomically $ STM.readTMVar threadRef
|
||||
metricsJ <- dumpReftechMetrics metrics
|
||||
candidatesJ <- dumpCandidates $ _mhCandidates lqHandler
|
||||
return $ J.object
|
||||
[ "key" J..= lq
|
||||
, "thread_id" J..= show (A.asyncThreadId threadId)
|
||||
, "alias" J..= _mhAlias lqHandler
|
||||
, "multiplexed_query" J..= Q.getQueryText (_mhQuery lqHandler)
|
||||
, "candidates" J..= candidatesJ
|
||||
, "metrics" J..= metricsJ
|
||||
]
|
||||
where
|
||||
dumpReftechMetrics metrics = do
|
||||
snapshotS <- Metrics.read $ _rmSnapshot metrics
|
||||
queryS <- Metrics.read $ _rmQuery metrics
|
||||
pushS <- Metrics.read $ _rmPush metrics
|
||||
totalS <- Metrics.read $ _rmTotal metrics
|
||||
return $ J.object
|
||||
[ "snapshot" J..= dumpStats snapshotS
|
||||
, "query" J..= dumpStats queryS
|
||||
, "push" J..= dumpStats pushS
|
||||
, "total" J..= dumpStats totalS
|
||||
]
|
||||
|
||||
dumpStats stats =
|
||||
J.object
|
||||
[ "mean" J..= Metrics.mean stats
|
||||
, "variance" J..= Metrics.variance stats
|
||||
, "count" J..= Metrics.count stats
|
||||
, "min" J..= Metrics.min stats
|
||||
, "max" J..= Metrics.max stats
|
||||
]
|
||||
dumpCandidates candidateMap = STM.atomically $ do
|
||||
candidates <- toListTMap candidateMap
|
||||
forM candidates $ \((usrVars, varVals), candidate) -> do
|
||||
candidateJ <- dumpCandidate candidate
|
||||
return $ J.object
|
||||
[ "session_vars" J..= usrVars
|
||||
, "variable_values" J..= varVals
|
||||
, "candidate" J..= candidateJ
|
||||
]
|
||||
dumpCandidate (CandidateState respId _ respTV curOps newOps) = do
|
||||
prevResHash <- STM.readTVar respTV
|
||||
curOpIds <- toListTMap curOps
|
||||
newOpIds <- toListTMap newOps
|
||||
return $ J.object
|
||||
[ "resp_id" J..= unRespId respId
|
||||
, "current_ops" J..= map fst curOpIds
|
||||
, "new_ops" J..= map fst newOpIds
|
||||
, "previous_result_hash" J..= prevResHash
|
||||
]
|
||||
|
||||
type ValidatedVariables = Map.HashMap G.Variable TxtEncodedPGVal
|
||||
|
||||
data LQHandler
|
||||
= LQHandler
|
||||
{ _mhAlias :: !G.Alias
|
||||
, _mhQuery :: !Q.Query
|
||||
, _mhCandidates ::
|
||||
!(TMap
|
||||
(UserVars, Maybe VariableValues)
|
||||
CandidateState
|
||||
)
|
||||
}
|
||||
|
||||
-- This type represents the state associated with
|
||||
-- the response of (role, gqlQueryText, userVars, variableValues)
|
||||
data CandidateState
|
||||
= CandidateState
|
||||
-- the laterally joined query responds with [(RespId, EncJSON)]
|
||||
-- so the resultid is used to determine the websockets
|
||||
-- where the data needs to be sent
|
||||
{ _csRespId :: !RespId
|
||||
|
||||
-- query variables which are validated and text encoded
|
||||
, _csValidatedVars :: !ValidatedVariables
|
||||
|
||||
-- we need to store the previous response
|
||||
, _csPrevRes :: !RespTV
|
||||
|
||||
-- the actions that have been run previously
|
||||
-- we run these if the response changes
|
||||
, _csCurOps :: !Sinks
|
||||
|
||||
-- we run these operations regardless
|
||||
-- and then merge them with current operations
|
||||
, _csNewOps :: !Sinks
|
||||
}
|
||||
|
||||
-- the multiplexed query associated with the livequery
|
||||
-- and the validated, text encoded query variables
|
||||
data MxOpCtx
|
||||
= MxOpCtx
|
||||
{ _mocGroup :: !LQGroup
|
||||
, _mocCandidate :: !CandidateId
|
||||
, _mocAlias :: !G.Alias
|
||||
, _mocQuery :: !Q.Query
|
||||
}
|
||||
|
||||
instance J.ToJSON MxOpCtx where
|
||||
toJSON (MxOpCtx lqGroup candidate als q) =
|
||||
J.object [ "query" J..= Q.getQueryText q
|
||||
, "alias" J..= als
|
||||
, "group" J..= lqGroup
|
||||
, "candidate" J..= candidate
|
||||
]
|
||||
|
||||
type MxOp = (MxOpCtx, ValidatedVariables)
|
||||
|
||||
mkMxOpCtx
|
||||
:: UserInfo -> GQLReqUnparsed
|
||||
-> G.Alias -> Q.Query
|
||||
-> MxOpCtx
|
||||
mkMxOpCtx userInfo req als query =
|
||||
MxOpCtx lqGroup candidateId als $ mkMxQuery query
|
||||
where
|
||||
candidateId = (userVars userInfo, _grVariables req)
|
||||
lqGroup = LQGroup (userRole userInfo) (_grQuery req)
|
||||
|
||||
mkMxQuery :: Q.Query -> Q.Query
|
||||
mkMxQuery baseQuery =
|
||||
Q.fromText $ mconcat $ map Q.getQueryText $
|
||||
[mxQueryPfx, baseQuery, mxQuerySfx]
|
||||
where
|
||||
mxQueryPfx :: Q.Query
|
||||
mxQueryPfx =
|
||||
[Q.sql|
|
||||
select
|
||||
_subs.result_id, _fld_resp.root as result
|
||||
from
|
||||
unnest(
|
||||
$1::uuid[], $2::json[]
|
||||
) _subs (result_id, result_vars)
|
||||
left outer join lateral
|
||||
(
|
||||
|]
|
||||
|
||||
mxQuerySfx :: Q.Query
|
||||
mxQuerySfx =
|
||||
[Q.sql|
|
||||
) _fld_resp ON ('true')
|
||||
|]
|
||||
|
||||
data LiveQueryId
|
||||
= LiveQueryId
|
||||
{ _lqiGroup :: !LQGroup
|
||||
, _lqiCandidate :: !CandidateId
|
||||
, _lqiSink :: !SinkId
|
||||
}
|
||||
|
||||
addLiveQuery
|
||||
:: PGExecCtx
|
||||
-> LiveQueriesState
|
||||
-- the query
|
||||
-> MxOp
|
||||
-- the action to be executed when result changes
|
||||
-> OnChange
|
||||
-> IO LiveQueryId
|
||||
addLiveQuery pgExecCtx lqState (mxOpCtx, valQVars) onResultAction = do
|
||||
|
||||
-- generate a new result id
|
||||
responseId <- newRespId
|
||||
|
||||
-- generate a new sink id
|
||||
sinkId <- newSinkId
|
||||
|
||||
-- a handler is returned only when it is newly created
|
||||
handlerM <- STM.atomically $ do
|
||||
handlerM <- STMMap.lookup handlerId lqMap
|
||||
case handlerM of
|
||||
Just (handler, _) -> do
|
||||
candidateM <- lookupTMap candidateId $ _mhCandidates handler
|
||||
case candidateM of
|
||||
Just candidate -> addToExistingCandidate sinkId candidate
|
||||
Nothing -> addToExistingHandler sinkId responseId handler
|
||||
return Nothing
|
||||
Nothing -> do
|
||||
handler <- newHandler sinkId responseId
|
||||
asyncRefTM <- STM.newEmptyTMVar
|
||||
STMMap.insert (handler, asyncRefTM) handlerId lqMap
|
||||
return $ Just (handler, asyncRefTM)
|
||||
|
||||
-- we can then attach a polling thread if it is new
|
||||
-- the livequery can only be cancelled after putTMVar
|
||||
onJust handlerM $ \(handler, pollerThreadTM) -> do
|
||||
metrics <- initRefetchMetrics
|
||||
threadRef <- A.async $ forever $ do
|
||||
pollQuery metrics batchSize pgExecCtx handler
|
||||
threadDelay $ refetchIntervalToMicro refetchInterval
|
||||
let threadState = ThreadState threadRef metrics
|
||||
STM.atomically $ STM.putTMVar pollerThreadTM threadState
|
||||
|
||||
return $ LiveQueryId handlerId candidateId sinkId
|
||||
|
||||
where
|
||||
|
||||
MxOpCtx handlerId candidateId als mxQuery = mxOpCtx
|
||||
LiveQueriesState lqOpts lqMap = lqState
|
||||
MxOpts batchSize refetchInterval = lqOpts
|
||||
|
||||
addToExistingCandidate sinkId handlerC =
|
||||
insertTMap onResultAction sinkId $ _csNewOps handlerC
|
||||
|
||||
newHandlerC sinkId responseId = do
|
||||
handlerC <- CandidateState
|
||||
responseId
|
||||
valQVars
|
||||
<$> STM.newTVar Nothing
|
||||
<*> newTMap
|
||||
<*> newTMap
|
||||
insertTMap onResultAction sinkId $ _csNewOps handlerC
|
||||
return handlerC
|
||||
|
||||
addToExistingHandler sinkId responseId handler = do
|
||||
handlerC <- newHandlerC sinkId responseId
|
||||
insertTMap handlerC candidateId $ _mhCandidates handler
|
||||
|
||||
newHandler sinkId responseId = do
|
||||
handler <- LQHandler als mxQuery <$> newTMap
|
||||
handlerC <- newHandlerC sinkId responseId
|
||||
insertTMap handlerC candidateId $ _mhCandidates handler
|
||||
return handler
|
||||
|
||||
type CandidateId = (UserVars, Maybe VariableValues)
|
||||
|
||||
removeLiveQuery
|
||||
:: LiveQueriesState
|
||||
-- the query and the associated operation
|
||||
-> LiveQueryId
|
||||
-> IO ()
|
||||
removeLiveQuery lqState (LiveQueryId handlerId candidateId sinkId) = do
|
||||
threadRefM <- STM.atomically $ do
|
||||
detM <- getQueryDet
|
||||
fmap join $ forM detM $
|
||||
\(handler, threadRef, candidate) ->
|
||||
cleanHandlerC (_mhCandidates handler) threadRef candidate
|
||||
onJust threadRefM (A.cancel . _tsThread)
|
||||
|
||||
where
|
||||
lqMap = _lqsLiveQueryMap lqState
|
||||
|
||||
getQueryDet = do
|
||||
handlerM <- STMMap.lookup handlerId lqMap
|
||||
fmap join $ forM handlerM $ \(handler, threadRef) -> do
|
||||
let LQHandler _ _ candidateMap = handler
|
||||
candidateM <- lookupTMap candidateId candidateMap
|
||||
return $ fmap (handler, threadRef,) candidateM
|
||||
|
||||
cleanHandlerC candidateMap threadRef handlerC = do
|
||||
let curOps = _csCurOps handlerC
|
||||
newOps = _csNewOps handlerC
|
||||
deleteTMap sinkId curOps
|
||||
deleteTMap sinkId newOps
|
||||
candidateIsEmpty <- (&&)
|
||||
<$> nullTMap curOps
|
||||
<*> nullTMap newOps
|
||||
when candidateIsEmpty $ deleteTMap candidateId candidateMap
|
||||
handlerIsEmpty <- nullTMap candidateMap
|
||||
-- when there is no need for handler
|
||||
-- i.e, this happens to be the last operation, take the
|
||||
-- ref for the polling thread to cancel it
|
||||
if handlerIsEmpty
|
||||
then do
|
||||
STMMap.delete handlerId lqMap
|
||||
Just <$> STM.takeTMVar threadRef
|
||||
else return Nothing
|
||||
|
||||
data CandidateSnapshot
|
||||
= CandidateSnapshot
|
||||
{ _csRespVars :: !RespVars
|
||||
, _csPrevResRef :: !RespTV
|
||||
, _csCurSinks :: ![OnChange]
|
||||
, _csNewSinks :: ![OnChange]
|
||||
}
|
||||
|
||||
pushCandidateResult :: GQResp -> Maybe RespHash -> CandidateSnapshot -> IO ()
|
||||
pushCandidateResult resp respHashM candidateSnapshot = do
|
||||
pushResultToSinks newSinks
|
||||
-- write to the current websockets if needed
|
||||
prevRespHashM <- STM.readTVarIO respRef
|
||||
when (isExecError resp || respHashM /= prevRespHashM) $ do
|
||||
pushResultToSinks curSinks
|
||||
STM.atomically $ STM.writeTVar respRef respHashM
|
||||
where
|
||||
CandidateSnapshot _ respRef curSinks newSinks = candidateSnapshot
|
||||
pushResultToSinks =
|
||||
A.mapConcurrently_ (\action -> action resp)
|
||||
|
||||
type RespVars = J.Value
|
||||
|
||||
newtype RespVarsList
|
||||
= RespVarsList { _unRespVarsList :: [RespVars]}
|
||||
|
||||
instance Q.ToPrepArg RespVarsList where
|
||||
toPrepVal (RespVarsList l) =
|
||||
Q.toPrepValHelper PTI.unknown encoder l
|
||||
where
|
||||
encoder =
|
||||
PE.array 114 . PE.dimensionArray foldl'
|
||||
(PE.encodingArray . PE.json_ast)
|
||||
|
||||
getRespVars :: UserVars -> ValidatedVariables -> RespVars
|
||||
getRespVars usrVars valVars =
|
||||
J.object [ "user" J..= usrVars
|
||||
, "variables" J..= fmap asJson valVars
|
||||
]
|
||||
where
|
||||
asJson = \case
|
||||
TENull -> J.Null
|
||||
TELit t -> J.String t
|
||||
|
||||
newtype BatchSize
|
||||
= BatchSize { unBatchSize :: Word32 }
|
||||
deriving (Show, Eq, J.ToJSON)
|
||||
|
||||
mkBatchSize :: Word32 -> BatchSize
|
||||
mkBatchSize = BatchSize
|
||||
|
||||
defaultBatchSize :: BatchSize
|
||||
defaultBatchSize =
|
||||
BatchSize 100
|
||||
|
||||
chunks :: Word32 -> [a] -> [[a]]
|
||||
chunks n =
|
||||
takeWhile (not.null) . unfoldr (Just . splitAt (fromIntegral n))
|
||||
|
||||
pollQuery
|
||||
:: RefetchMetrics
|
||||
-> BatchSize
|
||||
-> PGExecCtx
|
||||
-> LQHandler
|
||||
-> IO ()
|
||||
pollQuery metrics batchSize pgExecCtx handler = do
|
||||
|
||||
procInit <- Clock.getCurrentTime
|
||||
-- get a snapshot of all the candidates
|
||||
candidateSnapshotMap <- STM.atomically $ do
|
||||
candidates <- toListTMap candidateMap
|
||||
candidateSnapshots <- mapM getCandidateSnapshot candidates
|
||||
return $ Map.fromList candidateSnapshots
|
||||
let queryVarsBatches = chunks (unBatchSize batchSize) $
|
||||
getQueryVars candidateSnapshotMap
|
||||
|
||||
snapshotFinish <- Clock.getCurrentTime
|
||||
Metrics.add (_rmSnapshot metrics) $
|
||||
realToFrac $ Clock.diffUTCTime snapshotFinish procInit
|
||||
flip A.mapConcurrently_ queryVarsBatches $ \queryVars -> do
|
||||
queryInit <- Clock.getCurrentTime
|
||||
mxRes <- runExceptT $ runLazyTx' pgExecCtx $
|
||||
liftTx $ Q.listQE defaultTxErrorHandler
|
||||
pgQuery (mkMxQueryPrepArgs queryVars) True
|
||||
queryFinish <- Clock.getCurrentTime
|
||||
Metrics.add (_rmQuery metrics) $
|
||||
realToFrac $ Clock.diffUTCTime queryFinish queryInit
|
||||
let operations = getCandidateOperations candidateSnapshotMap mxRes
|
||||
-- concurrently push each unique result
|
||||
A.mapConcurrently_ (uncurry3 pushCandidateResult) operations
|
||||
pushFinish <- Clock.getCurrentTime
|
||||
Metrics.add (_rmPush metrics) $
|
||||
realToFrac $ Clock.diffUTCTime pushFinish queryFinish
|
||||
procFinish <- Clock.getCurrentTime
|
||||
Metrics.add (_rmTotal metrics) $
|
||||
realToFrac $ Clock.diffUTCTime procFinish procInit
|
||||
|
||||
where
|
||||
LQHandler alias pgQuery candidateMap = handler
|
||||
uncurry3 :: (a -> b -> c -> d) -> (a, b, c) -> d
|
||||
uncurry3 f (a, b, c) = f a b c
|
||||
|
||||
getCandidateSnapshot ((usrVars, _), handlerC) = do
|
||||
let CandidateState resId valVars respRef curOpsTV newOpsTV = handlerC
|
||||
curOpsL <- toListTMap curOpsTV
|
||||
newOpsL <- toListTMap newOpsTV
|
||||
forM_ newOpsL $ \(k, action) -> insertTMap action k curOpsTV
|
||||
resetTMap newOpsTV
|
||||
let resultVars = getRespVars usrVars valVars
|
||||
candidateSnapshot = CandidateSnapshot resultVars respRef
|
||||
(map snd curOpsL) (map snd newOpsL)
|
||||
return (resId, candidateSnapshot)
|
||||
|
||||
getQueryVars candidateSnapshotMap =
|
||||
Map.toList $ fmap _csRespVars candidateSnapshotMap
|
||||
|
||||
mkMxQueryPrepArgs l =
|
||||
let (respIdL, respVarL) = unzip l
|
||||
in (RespIdList respIdL, RespVarsList respVarL)
|
||||
|
||||
getCandidateOperations candidateSnapshotMap = \case
|
||||
Left e ->
|
||||
-- TODO: this is internal error
|
||||
let resp = GQExecError [encodeGQErr False e]
|
||||
in [ (resp, Nothing, snapshot)
|
||||
| (_, snapshot) <- Map.toList candidateSnapshotMap
|
||||
]
|
||||
Right responses ->
|
||||
flip mapMaybe responses $ \(respId, respEnc) ->
|
||||
-- TODO: change it to use bytestrings directly
|
||||
|
||||
let fldAlsT = G.unName $ G.unAlias alias
|
||||
respLbs = encJToLBS $ encJFromAssocList $
|
||||
pure $ (,) fldAlsT respEnc
|
||||
resp = GQSuccess respLbs
|
||||
respHash = mkRespHash respLbs
|
||||
in (resp, Just respHash,) <$> Map.lookup respId candidateSnapshotMap
|
115
server/src-lib/Hasura/GraphQL/Execute/LiveQuery/Types.hs
Normal file
115
server/src-lib/Hasura/GraphQL/Execute/LiveQuery/Types.hs
Normal file
@ -0,0 +1,115 @@
|
||||
module Hasura.GraphQL.Execute.LiveQuery.Types
|
||||
( OnChange
|
||||
, ThreadTM
|
||||
, SinkId
|
||||
, newSinkId
|
||||
, Sinks
|
||||
|
||||
, RespHash
|
||||
, mkRespHash
|
||||
, RespTV
|
||||
|
||||
, RefetchInterval
|
||||
, refetchIntervalFromMilli
|
||||
, refetchIntervalToMicro
|
||||
|
||||
, TMap
|
||||
, newTMap
|
||||
, resetTMap
|
||||
, nullTMap
|
||||
, insertTMap
|
||||
, deleteTMap
|
||||
, lookupTMap
|
||||
, toListTMap
|
||||
) where
|
||||
|
||||
import Data.Word (Word32)
|
||||
|
||||
import qualified Control.Concurrent.Async as A
|
||||
import qualified Control.Concurrent.STM as STM
|
||||
import qualified Crypto.Hash as CH
|
||||
import qualified Data.Aeson as J
|
||||
import qualified Data.ByteString.Lazy as LBS
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Data.UUID as UUID
|
||||
import qualified Data.UUID.V4 as UUID
|
||||
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.Prelude
|
||||
|
||||
type OnChange = GQResp -> IO ()
|
||||
type ThreadTM = STM.TMVar (A.Async ())
|
||||
|
||||
-- a cryptographic hash should ensure that
|
||||
-- a hash collision is almost improbable
|
||||
-- Blake2b because it is faster than Sha256
|
||||
-- With 256 bits, and 86400 * 365 (a subscription open for 365 days)
|
||||
-- there is ~ 4.294417×10-63 chance of a hash collision.
|
||||
|
||||
newtype RespHash
|
||||
= RespHash {unRespHash :: CH.Digest CH.Blake2b_256}
|
||||
deriving (Show, Eq)
|
||||
|
||||
instance J.ToJSON RespHash where
|
||||
toJSON = J.toJSON . show . unRespHash
|
||||
|
||||
mkRespHash :: LBS.ByteString -> RespHash
|
||||
mkRespHash = RespHash . CH.hashlazy
|
||||
|
||||
type RespTV = STM.TVar (Maybe RespHash)
|
||||
|
||||
newtype RefetchInterval
|
||||
= RefetchInterval {unRefetchInterval :: Word32}
|
||||
deriving (Show, Eq, J.ToJSON)
|
||||
|
||||
refetchIntervalFromMilli :: Word32 -> RefetchInterval
|
||||
refetchIntervalFromMilli = RefetchInterval
|
||||
|
||||
refetchIntervalToMicro :: RefetchInterval -> Int
|
||||
refetchIntervalToMicro ri = fromIntegral $ 1000 * unRefetchInterval ri
|
||||
|
||||
-- compared to stm.stmmap, this provides a much faster
|
||||
-- iteration over the elements at the cost of slower
|
||||
-- concurrent insertions
|
||||
newtype TMap k v
|
||||
= TMap {unTMap :: STM.TVar (Map.HashMap k v)}
|
||||
|
||||
newTMap :: STM.STM (TMap k v)
|
||||
newTMap =
|
||||
TMap <$> STM.newTVar Map.empty
|
||||
|
||||
resetTMap :: TMap k v -> STM.STM ()
|
||||
resetTMap =
|
||||
flip STM.writeTVar Map.empty . unTMap
|
||||
|
||||
nullTMap :: TMap k v -> STM.STM Bool
|
||||
nullTMap =
|
||||
fmap Map.null . STM.readTVar . unTMap
|
||||
|
||||
insertTMap :: (Eq k, Hashable k) => v -> k -> TMap k v -> STM.STM ()
|
||||
insertTMap v k mapTv =
|
||||
STM.modifyTVar' (unTMap mapTv) $ Map.insert k v
|
||||
|
||||
deleteTMap :: (Eq k, Hashable k) => k -> TMap k v -> STM.STM ()
|
||||
deleteTMap k mapTv =
|
||||
STM.modifyTVar' (unTMap mapTv) $ Map.delete k
|
||||
|
||||
lookupTMap :: (Eq k, Hashable k) => k -> TMap k v -> STM.STM (Maybe v)
|
||||
lookupTMap k =
|
||||
fmap (Map.lookup k) . STM.readTVar . unTMap
|
||||
|
||||
toListTMap :: TMap k v -> STM.STM [(k, v)]
|
||||
toListTMap =
|
||||
fmap Map.toList . STM.readTVar . unTMap
|
||||
|
||||
newtype SinkId
|
||||
= SinkId {_unSinkId :: UUID.UUID}
|
||||
deriving (Show, Eq, Hashable)
|
||||
|
||||
newSinkId :: IO SinkId
|
||||
newSinkId = SinkId <$> UUID.nextRandom
|
||||
|
||||
instance J.ToJSON SinkId where
|
||||
toJSON = J.toJSON . show
|
||||
|
||||
type Sinks = TMap SinkId OnChange
|
85
server/src-lib/Hasura/GraphQL/Execute/Plan.hs
Normal file
85
server/src-lib/Hasura/GraphQL/Execute/Plan.hs
Normal file
@ -0,0 +1,85 @@
|
||||
module Hasura.GraphQL.Execute.Plan
|
||||
( ReusablePlan(..)
|
||||
, PlanCache
|
||||
, getPlan
|
||||
, addPlan
|
||||
, initPlanCache
|
||||
, clearPlanCache
|
||||
, dumpPlanCache
|
||||
) where
|
||||
|
||||
import qualified Hasura.Cache as Cache
|
||||
import Hasura.Prelude
|
||||
|
||||
import qualified Data.Aeson as J
|
||||
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery as LQ
|
||||
import qualified Hasura.GraphQL.Execute.Query as EQ
|
||||
import qualified Hasura.GraphQL.Transport.HTTP.Protocol as GH
|
||||
import Hasura.RQL.Types
|
||||
|
||||
data PlanId
|
||||
= PlanId
|
||||
{ _piSchemaCacheVersion :: !SchemaCacheVer
|
||||
, _piRole :: !RoleName
|
||||
, _piOperationName :: !(Maybe GH.OperationName)
|
||||
, _piQuery :: !GH.GQLQueryText
|
||||
} deriving (Show, Eq, Generic)
|
||||
|
||||
instance Hashable PlanId
|
||||
|
||||
instance J.ToJSON PlanId where
|
||||
toJSON (PlanId scVer rn opNameM query) =
|
||||
J.object
|
||||
[ "schema_cache_version" J..= scVer
|
||||
, "role" J..= rn
|
||||
, "operation" J..= opNameM
|
||||
, "query" J..= query
|
||||
]
|
||||
|
||||
newtype PlanCache
|
||||
= PlanCache
|
||||
{ _unPlanCache :: Cache.UnboundedCache PlanId ReusablePlan
|
||||
}
|
||||
|
||||
data ReusablePlan
|
||||
= RPQuery !EQ.ReusableQueryPlan
|
||||
| RPSubs !LQ.SubsPlan
|
||||
|
||||
instance J.ToJSON ReusablePlan where
|
||||
toJSON = \case
|
||||
RPQuery queryPlan -> J.toJSON queryPlan
|
||||
RPSubs subsPlan -> J.toJSON subsPlan
|
||||
|
||||
initPlanCache :: IO PlanCache
|
||||
initPlanCache = PlanCache <$> Cache.initCache
|
||||
|
||||
getPlan
|
||||
:: SchemaCacheVer -> RoleName -> Maybe GH.OperationName -> GH.GQLQueryText
|
||||
-> PlanCache -> IO (Maybe ReusablePlan)
|
||||
getPlan schemaVer rn opNameM q (PlanCache planCache) =
|
||||
Cache.lookup planCache planId
|
||||
where
|
||||
planId = PlanId schemaVer rn opNameM q
|
||||
|
||||
addPlan
|
||||
:: SchemaCacheVer -> RoleName -> Maybe GH.OperationName -> GH.GQLQueryText
|
||||
-> ReusablePlan -> PlanCache -> IO ()
|
||||
addPlan schemaVer rn opNameM q queryPlan (PlanCache planCache) =
|
||||
Cache.insert planCache planId queryPlan
|
||||
where
|
||||
planId = PlanId schemaVer rn opNameM q
|
||||
|
||||
clearPlanCache :: PlanCache -> IO ()
|
||||
clearPlanCache (PlanCache planCache) =
|
||||
Cache.clearCache planCache
|
||||
|
||||
dumpPlanCache :: PlanCache -> IO J.Value
|
||||
dumpPlanCache (PlanCache cache) =
|
||||
J.toJSON <$> Cache.mapCache dumpEntry cache
|
||||
where
|
||||
dumpEntry (planId, plan) =
|
||||
J.object
|
||||
[ "id" J..= planId
|
||||
, "plan" J..= plan
|
||||
]
|
239
server/src-lib/Hasura/GraphQL/Execute/Query.hs
Normal file
239
server/src-lib/Hasura/GraphQL/Execute/Query.hs
Normal file
@ -0,0 +1,239 @@
|
||||
module Hasura.GraphQL.Execute.Query
|
||||
( convertQuerySelSet
|
||||
, queryOpFromPlan
|
||||
, ReusableQueryPlan
|
||||
) where
|
||||
|
||||
import Data.Has
|
||||
|
||||
import qualified Data.Aeson as J
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Data.HashSet as Set
|
||||
import qualified Data.IntMap as IntMap
|
||||
import qualified Data.TByteString as TBS
|
||||
import qualified Data.Text as T
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
import qualified Hasura.GraphQL.Resolve as R
|
||||
import qualified Hasura.GraphQL.Transport.HTTP.Protocol as GH
|
||||
import qualified Hasura.GraphQL.Validate as GV
|
||||
import qualified Hasura.GraphQL.Validate.Field as V
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Context
|
||||
import Hasura.GraphQL.Resolve.Context
|
||||
import Hasura.GraphQL.Validate.Types
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.DML.Select (asSingleRowJsonResp)
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
type PlanVariables = Map.HashMap G.Variable (Int, PGColType)
|
||||
type PrepArgMap = IntMap.IntMap Q.PrepArg
|
||||
|
||||
data PGPlan
|
||||
= PGPlan
|
||||
{ _ppQuery :: !Q.Query
|
||||
, _ppVariables :: !PlanVariables
|
||||
, _ppPrepared :: !PrepArgMap
|
||||
}
|
||||
|
||||
instance J.ToJSON PGPlan where
|
||||
toJSON (PGPlan q vars prepared) =
|
||||
J.object [ "query" J..= Q.getQueryText q
|
||||
, "variables" J..= vars
|
||||
, "prepared" J..= fmap show prepared
|
||||
]
|
||||
|
||||
data RootFieldPlan
|
||||
= RFPRaw !EncJSON
|
||||
| RFPPostgres !PGPlan
|
||||
|
||||
instance J.ToJSON RootFieldPlan where
|
||||
toJSON = \case
|
||||
RFPRaw encJson -> J.toJSON $ TBS.fromLBS $ encJToLBS encJson
|
||||
RFPPostgres pgPlan -> J.toJSON pgPlan
|
||||
|
||||
type VariableTypes = Map.HashMap G.Variable PGColType
|
||||
|
||||
data QueryPlan
|
||||
= QueryPlan
|
||||
{ _qpVariables :: ![G.VariableDefinition]
|
||||
, _qpFldPlans :: ![(G.Alias, RootFieldPlan)]
|
||||
}
|
||||
|
||||
data ReusableQueryPlan
|
||||
= ReusableQueryPlan
|
||||
{ _rqpVariableTypes :: !VariableTypes
|
||||
, _rqpFldPlans :: ![(G.Alias, RootFieldPlan)]
|
||||
}
|
||||
|
||||
instance J.ToJSON ReusableQueryPlan where
|
||||
toJSON (ReusableQueryPlan varTypes fldPlans) =
|
||||
J.object [ "variables" J..= show varTypes
|
||||
, "field_plans" J..= fldPlans
|
||||
]
|
||||
|
||||
getReusablePlan :: QueryPlan -> Maybe ReusableQueryPlan
|
||||
getReusablePlan (QueryPlan vars fldPlans) =
|
||||
if all fldPlanReusable $ map snd fldPlans
|
||||
then Just $ ReusableQueryPlan varTypes fldPlans
|
||||
else Nothing
|
||||
where
|
||||
allVars = Set.fromList $ map G._vdVariable vars
|
||||
|
||||
-- this is quite aggressive, we can improve this by
|
||||
-- computing used variables in each field
|
||||
allUsed fldPlanVars =
|
||||
Set.null $ Set.difference allVars $ Set.fromList fldPlanVars
|
||||
|
||||
fldPlanReusable = \case
|
||||
RFPRaw _ -> True
|
||||
RFPPostgres pgPlan -> allUsed $ Map.keys $ _ppVariables pgPlan
|
||||
|
||||
varTypesOfPlan = \case
|
||||
RFPRaw _ -> mempty
|
||||
RFPPostgres pgPlan -> snd <$> _ppVariables pgPlan
|
||||
|
||||
varTypes = Map.unions $ map (varTypesOfPlan . snd) fldPlans
|
||||
|
||||
withPlan
|
||||
:: UserVars -> PGPlan -> GV.AnnPGVarVals -> RespTx
|
||||
withPlan usrVars (PGPlan q reqVars prepMap) annVars = do
|
||||
prepMap' <- foldM getVar prepMap (Map.toList reqVars)
|
||||
let args = withUserVars usrVars $ IntMap.elems prepMap'
|
||||
asSingleRowJsonResp q args
|
||||
where
|
||||
getVar accum (var, (prepNo, _)) = do
|
||||
let varName = G.unName $ G.unVariable var
|
||||
(_, colVal) <- onNothing (Map.lookup var annVars) $
|
||||
throw500 $ "missing variable in annVars : " <> varName
|
||||
let prepVal = binEncoder colVal
|
||||
return $ IntMap.insert prepNo prepVal accum
|
||||
|
||||
-- turn the current plan into a transaction
|
||||
mkCurPlanTx
|
||||
:: UserVars
|
||||
-> QueryPlan
|
||||
-> LazyRespTx
|
||||
mkCurPlanTx usrVars (QueryPlan _ fldPlans) =
|
||||
fmap encJFromAssocList $ forM fldPlans $ \(alias, fldPlan) -> do
|
||||
fldResp <- case fldPlan of
|
||||
RFPRaw resp -> return resp
|
||||
RFPPostgres pgPlan -> liftTx $ planTx pgPlan
|
||||
return (G.unName $ G.unAlias alias, fldResp)
|
||||
where
|
||||
planTx (PGPlan q _ prepMap) =
|
||||
asSingleRowJsonResp q $ withUserVars usrVars $ IntMap.elems prepMap
|
||||
|
||||
withUserVars :: UserVars -> [Q.PrepArg] -> [Q.PrepArg]
|
||||
withUserVars usrVars l =
|
||||
Q.toPrepVal (Q.AltJ usrVars):l
|
||||
|
||||
data PlanningSt
|
||||
= PlanningSt
|
||||
{ _psArgNumber :: !Int
|
||||
, _psVariables :: !PlanVariables
|
||||
, _psPrepped :: !PrepArgMap
|
||||
}
|
||||
|
||||
initPlanningSt :: PlanningSt
|
||||
initPlanningSt =
|
||||
PlanningSt 2 Map.empty IntMap.empty
|
||||
|
||||
getVarArgNum
|
||||
:: (MonadState PlanningSt m)
|
||||
=> G.Variable -> PGColType -> m Int
|
||||
getVarArgNum var colTy = do
|
||||
PlanningSt curArgNum vars prepped <- get
|
||||
case Map.lookup var vars of
|
||||
Just argNum -> return $ fst argNum
|
||||
Nothing -> do
|
||||
put $ PlanningSt (curArgNum + 1)
|
||||
(Map.insert var (curArgNum, colTy) vars) prepped
|
||||
return curArgNum
|
||||
|
||||
addPrepArg
|
||||
:: (MonadState PlanningSt m)
|
||||
=> Int -> Q.PrepArg -> m ()
|
||||
addPrepArg argNum arg = do
|
||||
PlanningSt curArgNum vars prepped <- get
|
||||
put $ PlanningSt curArgNum vars $ IntMap.insert argNum arg prepped
|
||||
|
||||
getNextArgNum
|
||||
:: (MonadState PlanningSt m)
|
||||
=> m Int
|
||||
getNextArgNum = do
|
||||
PlanningSt curArgNum vars prepped <- get
|
||||
put $ PlanningSt (curArgNum + 1) vars prepped
|
||||
return curArgNum
|
||||
|
||||
prepareWithPlan
|
||||
:: (MonadState PlanningSt m)
|
||||
=> UnresolvedVal -> m S.SQLExp
|
||||
prepareWithPlan = \case
|
||||
R.UVPG annPGVal -> do
|
||||
let AnnPGVal varM isNullable colTy colVal = annPGVal
|
||||
argNum <- case (varM, isNullable) of
|
||||
(Just var, False) -> getVarArgNum var colTy
|
||||
_ -> getNextArgNum
|
||||
addPrepArg argNum $ binEncoder colVal
|
||||
return $ toPrepParam argNum colTy
|
||||
R.UVSessVar colTy sessVar ->
|
||||
return $ S.withTyAnn colTy $ withGeoVal colTy $
|
||||
S.SEOpApp (S.SQLOp "->>")
|
||||
[S.SEPrep 1, S.SELit $ T.toLower sessVar]
|
||||
R.UVSQL sqlExp -> return sqlExp
|
||||
|
||||
queryRootName :: Text
|
||||
queryRootName = "query_root"
|
||||
|
||||
convertQuerySelSet
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has TypeMap r
|
||||
, Has OpCtxMap r
|
||||
, Has FieldMap r
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
, Has UserInfo r
|
||||
)
|
||||
=> [G.VariableDefinition]
|
||||
-> V.SelSet
|
||||
-> m (LazyRespTx, Maybe ReusableQueryPlan)
|
||||
convertQuerySelSet varDefs fields = do
|
||||
usrVars <- asks (userVars . getter)
|
||||
fldPlans <- forM (toList fields) $ \fld -> do
|
||||
fldPlan <- case V._fName fld of
|
||||
"__type" -> RFPRaw . encJFromJValue <$> R.typeR fld
|
||||
"__schema" -> RFPRaw . encJFromJValue <$> R.schemaR fld
|
||||
"__typename" -> return $ RFPRaw $ encJFromJValue queryRootName
|
||||
_ -> do
|
||||
unresolvedAst <- R.queryFldToPGAST fld
|
||||
(q, PlanningSt _ vars prepped) <-
|
||||
flip runStateT initPlanningSt $ R.traverseQueryRootFldAST
|
||||
prepareWithPlan unresolvedAst
|
||||
return $ RFPPostgres $ PGPlan (R.toPGQuery q) vars prepped
|
||||
return (V._fAlias fld, fldPlan)
|
||||
let queryPlan = QueryPlan varDefs fldPlans
|
||||
reusablePlanM = getReusablePlan queryPlan
|
||||
return (mkCurPlanTx usrVars queryPlan, reusablePlanM)
|
||||
|
||||
-- use the existing plan and new variables to create a pg query
|
||||
queryOpFromPlan
|
||||
:: (MonadError QErr m)
|
||||
=> UserVars
|
||||
-> Maybe GH.VariableValues
|
||||
-> ReusableQueryPlan
|
||||
-> m LazyRespTx
|
||||
queryOpFromPlan usrVars varValsM (ReusableQueryPlan varTypes fldPlans) = do
|
||||
validatedVars <- GV.getAnnPGVarVals varTypes varValsM
|
||||
let tx = fmap encJFromAssocList $ forM fldPlans $ \(alias, fldPlan) -> do
|
||||
fldResp <- case fldPlan of
|
||||
RFPRaw resp -> return resp
|
||||
RFPPostgres pgPlan -> liftTx $ withPlan usrVars pgPlan validatedVars
|
||||
return (G.unName $ G.unAlias alias, fldResp)
|
||||
return tx
|
@ -3,15 +3,12 @@ module Hasura.GraphQL.Explain
|
||||
, GQLExplain
|
||||
) where
|
||||
|
||||
import Data.Has (getter)
|
||||
|
||||
import qualified Data.Aeson as J
|
||||
import qualified Data.Aeson.Casing as J
|
||||
import qualified Data.Aeson.TH as J
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
import qualified Text.Builder as TB
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Context
|
||||
@ -21,16 +18,17 @@ import Hasura.Prelude
|
||||
import Hasura.RQL.DML.Internal
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
import qualified Hasura.GraphQL.Execute as E
|
||||
import qualified Hasura.GraphQL.Resolve.Select as RS
|
||||
import qualified Hasura.GraphQL.Resolve as RS
|
||||
import qualified Hasura.GraphQL.Transport.HTTP.Protocol as GH
|
||||
import qualified Hasura.GraphQL.Validate as GV
|
||||
import qualified Hasura.RQL.DML.Select as RS
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
data GQLExplain
|
||||
= GQLExplain
|
||||
{ _gqeQuery :: !GH.GraphQLRequest
|
||||
{ _gqeQuery :: !GH.GQLReqParsed
|
||||
, _gqeUser :: !(Maybe (Map.HashMap Text Text))
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -47,15 +45,39 @@ data FieldPlan
|
||||
|
||||
$(J.deriveJSON (J.aesonDrop 3 J.camelCase) ''FieldPlan)
|
||||
|
||||
type Explain =
|
||||
(ReaderT (FieldMap, OrdByCtx, SQLGenCtx) (Except QErr))
|
||||
type Explain r =
|
||||
(ReaderT r (Except QErr))
|
||||
|
||||
runExplain
|
||||
:: (MonadError QErr m)
|
||||
=> (FieldMap, OrdByCtx, SQLGenCtx) -> Explain a -> m a
|
||||
=> r -> Explain r a -> m a
|
||||
runExplain ctx m =
|
||||
either throwError return $ runExcept $ runReaderT m ctx
|
||||
|
||||
resolveVal
|
||||
:: (MonadError QErr m)
|
||||
=> UserInfo -> UnresolvedVal -> m S.SQLExp
|
||||
resolveVal userInfo = \case
|
||||
RS.UVPG annPGVal ->
|
||||
txtConverter annPGVal
|
||||
RS.UVSessVar colTy sessVar -> do
|
||||
sessVarVal <- getSessVarVal userInfo sessVar
|
||||
return $ S.withTyAnn colTy $ withGeoVal colTy $
|
||||
S.SELit sessVarVal
|
||||
RS.UVSQL sqlExp -> return sqlExp
|
||||
|
||||
getSessVarVal
|
||||
:: (MonadError QErr m)
|
||||
=> UserInfo -> SessVar -> m SessVarVal
|
||||
getSessVarVal userInfo sessVar =
|
||||
onNothing (getVarVal sessVar usrVars) $
|
||||
throw400 UnexpectedPayload $
|
||||
"missing required session variable for role " <> rn <<>
|
||||
" : " <> sessVar
|
||||
where
|
||||
rn = userRole userInfo
|
||||
usrVars = userVars userInfo
|
||||
|
||||
explainField
|
||||
:: (MonadTx m)
|
||||
=> UserInfo -> GCtx -> SQLGenCtx -> Field -> m FieldPlan
|
||||
@ -65,28 +87,12 @@ explainField userInfo gCtx sqlGenCtx fld =
|
||||
"__schema" -> return $ FieldPlan fName Nothing Nothing
|
||||
"__typename" -> return $ FieldPlan fName Nothing Nothing
|
||||
_ -> do
|
||||
opCxt <- getOpCtx fName
|
||||
builderSQL <- runExplain (fldMap, orderByCtx, sqlGenCtx) $
|
||||
case opCxt of
|
||||
OCSelect (SelOpCtx tn hdrs permFilter permLimit) -> do
|
||||
validateHdrs hdrs
|
||||
toSQL . RS.mkSQLSelect False <$>
|
||||
RS.fromField txtConverter tn permFilter permLimit fld
|
||||
OCSelectPkey (SelPkOpCtx tn hdrs permFilter argMap) -> do
|
||||
validateHdrs hdrs
|
||||
toSQL . RS.mkSQLSelect True <$>
|
||||
RS.fromFieldByPKey txtConverter tn argMap permFilter fld
|
||||
OCSelectAgg (SelOpCtx tn hdrs permFilter permLimit) -> do
|
||||
validateHdrs hdrs
|
||||
toSQL . RS.mkAggSelect <$>
|
||||
RS.fromAggField txtConverter tn permFilter permLimit fld
|
||||
OCFuncQuery (FuncQOpCtx tn hdrs permFilter permLimit fn argSeq) ->
|
||||
procFuncQuery tn fn permFilter permLimit hdrs argSeq False
|
||||
OCFuncAggQuery (FuncQOpCtx tn hdrs permFilter permLimit fn argSeq) ->
|
||||
procFuncQuery tn fn permFilter permLimit hdrs argSeq True
|
||||
_ -> throw500 "unexpected mut field info for explain"
|
||||
|
||||
let txtSQL = TB.run builderSQL
|
||||
unresolvedAST <-
|
||||
runExplain (opCtxMap, userInfo, fldMap, orderByCtx, sqlGenCtx) $
|
||||
RS.queryFldToPGAST fld
|
||||
resolvedAST <- RS.traverseQueryRootFldAST (resolveVal userInfo)
|
||||
unresolvedAST
|
||||
let txtSQL = Q.getQueryText $ RS.toPGQuery resolvedAST
|
||||
withExplain = "EXPLAIN (FORMAT TEXT) " <> txtSQL
|
||||
planLines <- liftTx $ map runIdentity <$>
|
||||
Q.listQE dmlTxErrorHandler (Q.fromText withExplain) () True
|
||||
@ -98,44 +104,24 @@ explainField userInfo gCtx sqlGenCtx fld =
|
||||
fldMap = _gFields gCtx
|
||||
orderByCtx = _gOrdByCtx gCtx
|
||||
|
||||
getOpCtx f =
|
||||
onNothing (Map.lookup f opCtxMap) $ throw500 $
|
||||
"lookup failed: opctx: " <> showName f
|
||||
|
||||
procFuncQuery tn fn permFilter permLimit hdrs argSeq isAgg = do
|
||||
validateHdrs hdrs
|
||||
(tabArgs, eSel, frmItem) <-
|
||||
RS.fromFuncQueryField txtConverter fn argSeq isAgg fld
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ toSQL $
|
||||
RS.mkFuncSelectWith fn tn
|
||||
(RS.TablePerm permFilter permLimit) tabArgs strfyNum eSel frmItem
|
||||
|
||||
validateHdrs hdrs = do
|
||||
let receivedHdrs = userVars userInfo
|
||||
forM_ hdrs $ \hdr ->
|
||||
unless (isJust $ getVarVal hdr receivedHdrs) $
|
||||
throw400 NotFound $ hdr <<> " header is expected but not found"
|
||||
|
||||
explainGQLQuery
|
||||
:: (MonadError QErr m, MonadIO m)
|
||||
=> Q.PGPool
|
||||
-> Q.TxIsolation
|
||||
=> PGExecCtx
|
||||
-> SchemaCache
|
||||
-> SQLGenCtx
|
||||
-> GQLExplain
|
||||
-> m EncJSON
|
||||
explainGQLQuery pool iso sc sqlGenCtx (GQLExplain query userVarsRaw)= do
|
||||
execPlan <- E.getExecPlan userInfo sc query
|
||||
explainGQLQuery pgExecCtx sc sqlGenCtx (GQLExplain query userVarsRaw)= do
|
||||
execPlan <- E.getExecPlanPartial userInfo sc query
|
||||
(gCtx, rootSelSet) <- case execPlan of
|
||||
E.GExPHasura gCtx rootSelSet ->
|
||||
E.GExPHasura (gCtx, rootSelSet, _) ->
|
||||
return (gCtx, rootSelSet)
|
||||
E.GExPRemote _ _ ->
|
||||
throw400 InvalidParams "only hasura queries can be explained"
|
||||
case rootSelSet of
|
||||
GV.RQuery selSet -> do
|
||||
let tx = mapM (explainField userInfo gCtx sqlGenCtx) (toList selSet)
|
||||
plans <- liftIO (runExceptT $ runTx tx) >>= liftEither
|
||||
plans <- liftIO (runExceptT $ runLazyTx pgExecCtx tx) >>= liftEither
|
||||
return $ encJFromJValue plans
|
||||
GV.RMutation _ ->
|
||||
throw400 InvalidParams "only queries can be explained"
|
||||
@ -144,5 +130,3 @@ explainGQLQuery pool iso sc sqlGenCtx (GQLExplain query userVarsRaw)= do
|
||||
where
|
||||
usrVars = mkUserVars $ maybe [] Map.toList userVarsRaw
|
||||
userInfo = mkUserInfo (fromMaybe adminRole $ roleFromVars usrVars) usrVars
|
||||
|
||||
runTx tx = runLazyTx pool iso $ withUserInfo userInfo tx
|
||||
|
@ -1,20 +1,29 @@
|
||||
module Hasura.GraphQL.Resolve
|
||||
( resolveQuerySelSet
|
||||
, resolveMutSelSet
|
||||
, resolveSubsFld
|
||||
( mutFldToTx
|
||||
, queryFldToPGAST
|
||||
, RS.traverseQueryRootFldAST
|
||||
, RS.toPGQuery
|
||||
, UnresolvedVal(..)
|
||||
, AnnPGVal(..)
|
||||
, RS.QueryRootFldUnresolved
|
||||
, resolveValPrep
|
||||
, queryFldToSQL
|
||||
, schemaR
|
||||
, typeR
|
||||
) where
|
||||
|
||||
import Hasura.Prelude
|
||||
import Data.Has
|
||||
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Context
|
||||
import Hasura.GraphQL.Resolve.Context
|
||||
import Hasura.GraphQL.Resolve.Introspect
|
||||
import Hasura.GraphQL.Validate.Field
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.DML.Internal (sessVarFromCurrentSetting)
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
|
||||
@ -30,15 +39,17 @@ validateHdrs userInfo hdrs = do
|
||||
unless (isJust $ getVarVal hdr receivedVars) $
|
||||
throw400 NotFound $ hdr <<> " header is expected but not found"
|
||||
|
||||
resolvePGFld
|
||||
:: UserInfo
|
||||
-> GCtx
|
||||
-> SQLGenCtx
|
||||
-> Field
|
||||
-> Q.TxE QErr EncJSON
|
||||
resolvePGFld userInfo gCtx sqlCtx fld = do
|
||||
opCxt <- getOpCtx $ _fName fld
|
||||
join $ runConvert (fldMap, orderByCtx, insCtxMap, sqlCtx) $ case opCxt of
|
||||
queryFldToPGAST
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r, Has UserInfo r
|
||||
, Has OpCtxMap r
|
||||
)
|
||||
=> Field
|
||||
-> m RS.QueryRootFldUnresolved
|
||||
queryFldToPGAST fld = do
|
||||
opCtx <- getOpCtx $ _fName fld
|
||||
userInfo <- asks getter
|
||||
case opCtx of
|
||||
OCSelect ctx -> do
|
||||
validateHdrs userInfo (_socHeaders ctx)
|
||||
RS.convertSelect ctx fld
|
||||
@ -50,11 +61,51 @@ resolvePGFld userInfo gCtx sqlCtx fld = do
|
||||
RS.convertAggSelect ctx fld
|
||||
OCFuncQuery ctx -> do
|
||||
validateHdrs userInfo (_fqocHeaders ctx)
|
||||
RS.convertFuncQuery ctx False fld
|
||||
RS.convertFuncQuerySimple ctx fld
|
||||
OCFuncAggQuery ctx -> do
|
||||
validateHdrs userInfo (_fqocHeaders ctx)
|
||||
RS.convertFuncQuery ctx True fld
|
||||
RS.convertFuncQueryAgg ctx fld
|
||||
OCInsert _ ->
|
||||
throw500 "unexpected OCInsert for query field context"
|
||||
OCUpdate _ ->
|
||||
throw500 "unexpected OCUpdate for query field context"
|
||||
OCDelete _ ->
|
||||
throw500 "unexpected OCDelete for query field context"
|
||||
|
||||
queryFldToSQL
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r, Has UserInfo r
|
||||
, Has OpCtxMap r
|
||||
)
|
||||
=> PrepFn m
|
||||
-> Field
|
||||
-> m Q.Query
|
||||
queryFldToSQL fn fld = do
|
||||
pgAST <- queryFldToPGAST fld
|
||||
resolvedAST <- flip RS.traverseQueryRootFldAST pgAST $ \case
|
||||
UVPG annPGVal -> fn annPGVal
|
||||
UVSQL sqlExp -> return sqlExp
|
||||
UVSessVar colTy sessVar -> sessVarFromCurrentSetting colTy sessVar
|
||||
return $ RS.toPGQuery resolvedAST
|
||||
|
||||
mutFldToTx
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has UserInfo r
|
||||
, Has OpCtxMap r
|
||||
, Has FieldMap r
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
, Has InsCtxMap r
|
||||
)
|
||||
=> Field
|
||||
-> m RespTx
|
||||
mutFldToTx fld = do
|
||||
userInfo <- asks getter
|
||||
opCtx <- getOpCtx $ _fName fld
|
||||
case opCtx of
|
||||
OCInsert ctx -> do
|
||||
let roleName = userRole userInfo
|
||||
validateHdrs userInfo (_iocHeaders ctx)
|
||||
RI.convertInsert roleName (_iocTable ctx) fld
|
||||
OCUpdate ctx -> do
|
||||
@ -63,65 +114,24 @@ resolvePGFld userInfo gCtx sqlCtx fld = do
|
||||
OCDelete ctx -> do
|
||||
validateHdrs userInfo (_docHeaders ctx)
|
||||
RM.convertDelete ctx fld
|
||||
where
|
||||
opCtxMap = _gOpCtxMap gCtx
|
||||
fldMap = _gFields gCtx
|
||||
orderByCtx = _gOrdByCtx gCtx
|
||||
insCtxMap = _gInsCtxMap gCtx
|
||||
roleName = userRole userInfo
|
||||
OCSelect _ ->
|
||||
throw500 "unexpected query field context for a mutation field"
|
||||
OCSelectPkey _ ->
|
||||
throw500 "unexpected query field context for a mutation field"
|
||||
OCSelectAgg _ ->
|
||||
throw500 "unexpected query field context for a mutation field"
|
||||
OCFuncQuery _ ->
|
||||
throw500 "unexpected query field context for a mutation field"
|
||||
OCFuncAggQuery _ ->
|
||||
throw500 "unexpected query field context for a mutation field"
|
||||
|
||||
getOpCtx f =
|
||||
onNothing (Map.lookup f opCtxMap) $ throw500 $
|
||||
"lookup failed: opctx: " <> showName f
|
||||
|
||||
mkRootTypeName :: G.OperationType -> Text
|
||||
mkRootTypeName = \case
|
||||
G.OperationTypeQuery -> "query_root"
|
||||
G.OperationTypeMutation -> "mutation_root"
|
||||
G.OperationTypeSubscription -> "subscription_root"
|
||||
|
||||
resolveQuerySelSet
|
||||
:: (MonadTx m)
|
||||
=> UserInfo
|
||||
-> GCtx
|
||||
-> SQLGenCtx
|
||||
-> SelSet
|
||||
-> m EncJSON
|
||||
resolveQuerySelSet userInfo gCtx sqlGenCtx fields =
|
||||
fmap encJFromAssocList $ forM (toList fields) $ \fld -> do
|
||||
fldResp <- case _fName fld of
|
||||
"__type" -> encJFromJValue <$> runReaderT (typeR fld) gCtx
|
||||
"__schema" -> encJFromJValue <$> runReaderT (schemaR fld) gCtx
|
||||
"__typename" -> return $ encJFromJValue $
|
||||
mkRootTypeName G.OperationTypeQuery
|
||||
_ -> liftTx $ resolvePGFld userInfo gCtx sqlGenCtx fld
|
||||
return (G.unName $ G.unAlias $ _fAlias fld, fldResp)
|
||||
|
||||
resolveMutSelSet
|
||||
:: (MonadTx m)
|
||||
=> UserInfo
|
||||
-> GCtx
|
||||
-> SQLGenCtx
|
||||
-> SelSet
|
||||
-> m EncJSON
|
||||
resolveMutSelSet userInfo gCtx sqlGenCtx fields =
|
||||
fmap encJFromAssocList $ forM (toList fields) $ \fld -> do
|
||||
fldResp <- case _fName fld of
|
||||
"__typename" -> return $ encJFromJValue $
|
||||
mkRootTypeName G.OperationTypeMutation
|
||||
_ -> liftTx $ resolvePGFld userInfo gCtx sqlGenCtx fld
|
||||
return (G.unName $ G.unAlias $ _fAlias fld, fldResp)
|
||||
|
||||
resolveSubsFld
|
||||
:: (MonadTx m)
|
||||
=> UserInfo
|
||||
-> GCtx
|
||||
-> SQLGenCtx
|
||||
-> Field
|
||||
-> m EncJSON
|
||||
resolveSubsFld userInfo gCtx sqlGenCtx fld = do
|
||||
resp <- case _fName fld of
|
||||
"__typename" -> return $ encJFromJValue $
|
||||
mkRootTypeName G.OperationTypeSubscription
|
||||
_ -> liftTx $ resolvePGFld userInfo gCtx sqlGenCtx fld
|
||||
return $ encJFromAssocList [(G.unName $ G.unAlias $ _fAlias fld, resp)]
|
||||
getOpCtx
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has OpCtxMap r
|
||||
)
|
||||
=> G.Name -> m OpCtx
|
||||
getOpCtx f = do
|
||||
opCtxMap <- asks getter
|
||||
onNothing (Map.lookup f opCtxMap) $ throw500 $
|
||||
"lookup failed: opctx: " <> showName f
|
||||
|
@ -1,12 +1,10 @@
|
||||
module Hasura.GraphQL.Resolve.BoolExp
|
||||
( parseBoolExp
|
||||
, pgColValToBoolExp
|
||||
) where
|
||||
|
||||
import Data.Has
|
||||
import Hasura.Prelude
|
||||
|
||||
import qualified Data.HashMap.Strict as Map
|
||||
import qualified Data.HashMap.Strict.InsOrd as OMap
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
@ -24,7 +22,7 @@ parseOpExps
|
||||
=> PGColType -> AnnInpVal -> m [OpExp]
|
||||
parseOpExps colTy annVal = do
|
||||
opExpsM <- flip withObjectM annVal $ \nt objM -> forM objM $ \obj ->
|
||||
forM (OMap.toList obj) $ \(k, v) -> do
|
||||
forM (OMap.toList obj) $ \(k, v) ->
|
||||
case k of
|
||||
"_eq" -> fmap (AEQ True) <$> asPGColValM v
|
||||
"_ne" -> fmap (ANE True) <$> asPGColValM v
|
||||
@ -93,59 +91,42 @@ parseOpExps colTy annVal = do
|
||||
throw500 "expected \"use_spheroid\" input field in st_d_within"
|
||||
useSpheroid <- asPGColVal useSpheroidVal
|
||||
return $ ASTDWithinGeog $ DWithinGeogOp dist from useSpheroid
|
||||
PGGeometry -> do
|
||||
PGGeometry ->
|
||||
return $ ASTDWithinGeom $ DWithinGeomOp dist from
|
||||
_ -> throw500 "expected PGGeometry/PGGeography column for st_d_within"
|
||||
|
||||
parseAsEqOp
|
||||
:: (MonadError QErr m)
|
||||
=> AnnInpVal -> m [OpExp]
|
||||
parseAsEqOp annVal = do
|
||||
annValOpExp <- AEQ True <$> asPGColVal annVal
|
||||
return [annValOpExp]
|
||||
|
||||
parseColExp
|
||||
:: (MonadError QErr m, MonadReader r m, Has FieldMap r)
|
||||
=> PrepFn m -> G.NamedType -> G.Name -> AnnInpVal
|
||||
-> m AnnBoolExpFldSQL
|
||||
parseColExp f nt n val = do
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has FieldMap r
|
||||
)
|
||||
=> G.NamedType -> G.Name -> AnnInpVal
|
||||
-> m (AnnBoolExpFld UnresolvedVal)
|
||||
parseColExp nt n val = do
|
||||
fldInfo <- getFldInfo nt n
|
||||
case fldInfo of
|
||||
Left pgColInfo -> do
|
||||
opExps <- parseOpExps (pgiType pgColInfo) val
|
||||
AVCol pgColInfo <$> traverse (traverse f) opExps
|
||||
return $ AVCol pgColInfo $ map (fmap UVPG) opExps
|
||||
Right (relInfo, _, permExp, _) -> do
|
||||
relBoolExp <- parseBoolExp f val
|
||||
return $ AVRel relInfo $ andAnnBoolExps relBoolExp permExp
|
||||
relBoolExp <- parseBoolExp val
|
||||
return $ AVRel relInfo $ andAnnBoolExps relBoolExp $
|
||||
fmapAnnBoolExp partialSQLExpToUnresolvedVal permExp
|
||||
|
||||
parseBoolExp
|
||||
:: (MonadError QErr m, MonadReader r m, Has FieldMap r)
|
||||
=> PrepFn m -> AnnInpVal -> m AnnBoolExpSQL
|
||||
parseBoolExp f annGVal = do
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has FieldMap r
|
||||
)
|
||||
=> AnnInpVal -> m (AnnBoolExp UnresolvedVal)
|
||||
parseBoolExp annGVal = do
|
||||
boolExpsM <-
|
||||
flip withObjectM annGVal
|
||||
$ \nt objM -> forM objM $ \obj -> forM (OMap.toList obj) $ \(k, v) -> if
|
||||
| k == "_or" -> BoolOr . fromMaybe []
|
||||
<$> parseMany (parseBoolExp f) v
|
||||
<$> parseMany parseBoolExp v
|
||||
| k == "_and" -> BoolAnd . fromMaybe []
|
||||
<$> parseMany (parseBoolExp f) v
|
||||
| k == "_not" -> BoolNot <$> parseBoolExp f v
|
||||
| otherwise -> BoolFld <$> parseColExp f nt k v
|
||||
<$> parseMany parseBoolExp v
|
||||
| k == "_not" -> BoolNot <$> parseBoolExp v
|
||||
| otherwise -> BoolFld <$> parseColExp nt k v
|
||||
return $ BoolAnd $ fromMaybe [] boolExpsM
|
||||
|
||||
type PGColValMap = Map.HashMap G.Name AnnInpVal
|
||||
|
||||
pgColValToBoolExp
|
||||
:: (MonadError QErr m)
|
||||
=> PrepFn m -> PGColArgMap -> PGColValMap -> m AnnBoolExpSQL
|
||||
pgColValToBoolExp f colArgMap colValMap = do
|
||||
colExps <- forM colVals $ \(name, val) ->
|
||||
BoolFld <$> do
|
||||
opExps <- parseAsEqOp val
|
||||
colInfo <- onNothing (Map.lookup name colArgMap) $
|
||||
throw500 $ "column name " <> showName name
|
||||
<> " not found in column arguments map"
|
||||
AVCol colInfo <$> traverse (traverse f) opExps
|
||||
return $ BoolAnd colExps
|
||||
where
|
||||
colVals = Map.toList colValMap
|
||||
|
@ -14,6 +14,11 @@ module Hasura.GraphQL.Resolve.Context
|
||||
, LazyRespTx
|
||||
, AnnPGVal(..)
|
||||
, PrepFn
|
||||
, UnresolvedVal(..)
|
||||
, resolveValPrep
|
||||
, resolveValTxt
|
||||
, AnnBoolExpUnresolved
|
||||
, partialSQLExpToUnresolvedVal
|
||||
, InsertTxConflictCtx(..)
|
||||
, getFldInfo
|
||||
, getPGColInfo
|
||||
@ -21,13 +26,16 @@ module Hasura.GraphQL.Resolve.Context
|
||||
, withArg
|
||||
, withArgM
|
||||
, nameAsPath
|
||||
|
||||
, PrepArgs
|
||||
, Convert
|
||||
, runConvert
|
||||
, withPrepArgs
|
||||
, prepare
|
||||
, prepareColVal
|
||||
, withPrepArgs
|
||||
|
||||
, txtConverter
|
||||
|
||||
, withSelSet
|
||||
, fieldAsPath
|
||||
, module Hasura.GraphQL.Utils
|
||||
) where
|
||||
|
||||
@ -44,7 +52,6 @@ import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
import Hasura.GraphQL.Resolve.ContextTypes
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Utils
|
||||
import Hasura.GraphQL.Validate.Field
|
||||
import Hasura.GraphQL.Validate.Types
|
||||
@ -52,6 +59,8 @@ import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
import Hasura.RQL.DML.Internal (sessVarFromCurrentSetting)
|
||||
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
data InsResp
|
||||
@ -61,12 +70,6 @@ data InsResp
|
||||
} deriving (Show, Eq)
|
||||
$(J.deriveJSON (J.aesonDrop 3 J.snakeCase) ''InsResp)
|
||||
|
||||
type RespTx = Q.TxE QErr EncJSON
|
||||
|
||||
type LazyRespTx = LazyTx QErr EncJSON
|
||||
|
||||
type PrepFn m = AnnPGVal -> m S.SQLExp
|
||||
|
||||
data AnnPGVal
|
||||
= AnnPGVal
|
||||
{ _apvVariable :: !(Maybe G.Variable)
|
||||
@ -75,10 +78,30 @@ data AnnPGVal
|
||||
, _apvValue :: !PGColValue
|
||||
} deriving (Show, Eq)
|
||||
|
||||
type PrepFn m = AnnPGVal -> m S.SQLExp
|
||||
|
||||
-- lifts PartialSQLExp to UnresolvedVal
|
||||
partialSQLExpToUnresolvedVal :: PartialSQLExp -> UnresolvedVal
|
||||
partialSQLExpToUnresolvedVal = \case
|
||||
PSESessVar colTy sessVar -> UVSessVar colTy sessVar
|
||||
PSESQLExp s -> UVSQL s
|
||||
|
||||
-- A value that will be converted to an sql expression eventually
|
||||
data UnresolvedVal
|
||||
-- From a session variable
|
||||
= UVSessVar !PGColType !SessVar
|
||||
-- This is postgres
|
||||
| UVPG !AnnPGVal
|
||||
-- This is a full resolved sql expression
|
||||
| UVSQL !S.SQLExp
|
||||
deriving (Show, Eq)
|
||||
|
||||
type AnnBoolExpUnresolved = AnnBoolExp UnresolvedVal
|
||||
|
||||
getFldInfo
|
||||
:: (MonadError QErr m, MonadReader r m, Has FieldMap r)
|
||||
=> G.NamedType -> G.Name
|
||||
-> m (Either PGColInfo (RelInfo, Bool, AnnBoolExpSQL, Maybe Int))
|
||||
-> m (Either PGColInfo (RelInfo, Bool, AnnBoolExpPartialSQL, Maybe Int))
|
||||
getFldInfo nt n = do
|
||||
fldMap <- asks getter
|
||||
onNothing (Map.lookup (nt,n) fldMap) $
|
||||
@ -138,19 +161,28 @@ withArgM args arg f = prependArgsInPath $ nameAsPath arg $
|
||||
|
||||
type PrepArgs = Seq.Seq Q.PrepArg
|
||||
|
||||
type Convert =
|
||||
(ReaderT ( FieldMap
|
||||
, OrdByCtx
|
||||
, InsCtxMap
|
||||
, SQLGenCtx
|
||||
) (Except QErr)
|
||||
)
|
||||
|
||||
prepare
|
||||
:: (MonadState PrepArgs m) => PrepFn m
|
||||
:: (MonadState PrepArgs m) => AnnPGVal -> m S.SQLExp
|
||||
prepare (AnnPGVal _ _ colTy colVal) =
|
||||
prepareColVal colTy colVal
|
||||
|
||||
resolveValPrep
|
||||
:: (MonadState PrepArgs m)
|
||||
=> UnresolvedVal -> m S.SQLExp
|
||||
resolveValPrep = \case
|
||||
UVPG annPGVal -> prepare annPGVal
|
||||
UVSessVar colTy sessVar -> sessVarFromCurrentSetting colTy sessVar
|
||||
UVSQL sqlExp -> return sqlExp
|
||||
|
||||
resolveValTxt :: (Applicative f) => UnresolvedVal -> f S.SQLExp
|
||||
resolveValTxt = \case
|
||||
UVPG annPGVal -> txtConverter annPGVal
|
||||
UVSessVar colTy sessVar -> sessVarFromCurrentSetting colTy sessVar
|
||||
UVSQL sqlExp -> pure sqlExp
|
||||
|
||||
withPrepArgs :: StateT PrepArgs m a -> m (a, PrepArgs)
|
||||
withPrepArgs m = runStateT m Seq.empty
|
||||
|
||||
prepareColVal
|
||||
:: (MonadState PrepArgs m)
|
||||
=> PGColType -> PGColValue -> m S.SQLExp
|
||||
@ -159,19 +191,15 @@ prepareColVal colTy colVal = do
|
||||
put (preparedArgs Seq.|> binEncoder colVal)
|
||||
return $ toPrepParam (Seq.length preparedArgs + 1) colTy
|
||||
|
||||
|
||||
txtConverter :: Monad m => PrepFn m
|
||||
txtConverter :: Applicative f => AnnPGVal -> f S.SQLExp
|
||||
txtConverter (AnnPGVal _ _ a b) =
|
||||
return $ toTxtValue a b
|
||||
pure $ toTxtValue a b
|
||||
|
||||
withPrepArgs :: StateT PrepArgs Convert a -> Convert (a, PrepArgs)
|
||||
withPrepArgs m = runStateT m Seq.empty
|
||||
withSelSet :: (Monad m) => SelSet -> (Field -> m a) -> m [(Text, a)]
|
||||
withSelSet selSet f =
|
||||
forM (toList selSet) $ \fld -> do
|
||||
res <- f fld
|
||||
return (G.unName $ G.unAlias $ _fAlias fld, res)
|
||||
|
||||
runConvert
|
||||
:: (MonadError QErr m)
|
||||
=> (FieldMap, OrdByCtx, InsCtxMap, SQLGenCtx)
|
||||
-> Convert a
|
||||
-> m a
|
||||
runConvert ctx m =
|
||||
either throwError return $
|
||||
runExcept $ runReaderT m ctx
|
||||
fieldAsPath :: (MonadError QErr m) => Field -> m a -> m a
|
||||
fieldAsPath = nameAsPath . _fName
|
||||
|
@ -13,13 +13,13 @@ import Hasura.SQL.Types
|
||||
|
||||
type FieldMap
|
||||
= Map.HashMap (G.NamedType, G.Name)
|
||||
(Either PGColInfo (RelInfo, Bool, AnnBoolExpSQL, Maybe Int))
|
||||
(Either PGColInfo (RelInfo, Bool, AnnBoolExpPartialSQL, Maybe Int))
|
||||
|
||||
-- order by context
|
||||
data OrdByItem
|
||||
= OBIPGCol !PGColInfo
|
||||
| OBIRel !RelInfo !AnnBoolExpSQL
|
||||
| OBIAgg !RelInfo !AnnBoolExpSQL
|
||||
| OBIRel !RelInfo !AnnBoolExpPartialSQL
|
||||
| OBIAgg !RelInfo !AnnBoolExpPartialSQL
|
||||
deriving (Show, Eq)
|
||||
|
||||
type OrdByItemMap = Map.HashMap G.Name OrdByItem
|
||||
@ -38,15 +38,15 @@ type RelationInfoMap = Map.HashMap RelName RelInfo
|
||||
data UpdPermForIns
|
||||
= UpdPermForIns
|
||||
{ upfiCols :: ![PGCol]
|
||||
, upfiFilter :: !AnnBoolExpSQL
|
||||
, upfiSet :: !PreSetCols
|
||||
, upfiFilter :: !AnnBoolExpPartialSQL
|
||||
, upfiSet :: !PreSetColsPartial
|
||||
} deriving (Show, Eq)
|
||||
|
||||
data InsCtx
|
||||
= InsCtx
|
||||
{ icView :: !QualifiedTable
|
||||
, icAllCols :: ![PGColInfo]
|
||||
, icSet :: !PreSetCols
|
||||
, icSet :: !PreSetColsPartial
|
||||
, icRelations :: !RelationInfoMap
|
||||
, icUpdPerm :: !(Maybe UpdPermForIns)
|
||||
} deriving (Show, Eq)
|
||||
|
@ -29,7 +29,10 @@ import Hasura.GraphQL.Resolve.Mutation
|
||||
import Hasura.GraphQL.Resolve.Select
|
||||
import Hasura.GraphQL.Validate.Field
|
||||
import Hasura.GraphQL.Validate.Types
|
||||
import Hasura.RQL.DML.Internal (dmlTxErrorHandler)
|
||||
import Hasura.RQL.DML.Internal ( dmlTxErrorHandler
|
||||
, convPartialSQLExp
|
||||
, sessVarFromCurrentSetting
|
||||
)
|
||||
import Hasura.RQL.DML.Mutation
|
||||
import Hasura.RQL.GBoolExp (toSQLBoolExp)
|
||||
import Hasura.RQL.Types
|
||||
@ -121,13 +124,15 @@ traverseInsObj rim (gName, annVal) defVal@(AnnInsObj cols objRels arrRels) =
|
||||
|
||||
let rTable = riRTable relInfo
|
||||
InsCtx rtView rtCols rtDefVals rtRelInfoMap rtUpdPerm <- getInsCtx rTable
|
||||
rtDefValsRes <- mapM (convPartialSQLExp sessVarFromCurrentSetting)
|
||||
rtDefVals
|
||||
|
||||
withPathK (G.unName gName) $ case riType relInfo of
|
||||
ObjRel -> do
|
||||
dataObj <- asObject dataVal
|
||||
annDataObj <- mkAnnInsObj rtRelInfoMap dataObj
|
||||
ccM <- forM onConflictM $ parseOnConflict rTable rtUpdPerm
|
||||
let singleObjIns = AnnIns annDataObj ccM rtView rtCols rtDefVals
|
||||
let singleObjIns = AnnIns annDataObj ccM rtView rtCols rtDefValsRes
|
||||
objRelIns = RelIns singleObjIns relInfo
|
||||
return (AnnInsObj cols (objRelIns:objRels) arrRels)
|
||||
|
||||
@ -138,7 +143,8 @@ traverseInsObj rim (gName, annVal) defVal@(AnnInsObj cols objRels arrRels) =
|
||||
dataObj <- asObject arrDataVal
|
||||
mkAnnInsObj rtRelInfoMap dataObj
|
||||
ccM <- forM onConflictM $ parseOnConflict rTable rtUpdPerm
|
||||
let multiObjIns = AnnIns annDataObjs ccM rtView rtCols rtDefVals
|
||||
let multiObjIns = AnnIns annDataObjs ccM rtView
|
||||
rtCols rtDefValsRes
|
||||
arrRelIns = RelIns multiObjIns relInfo
|
||||
return (AnnInsObj cols objRels (arrRelIns:arrRels))
|
||||
-- if array relation insert input data has empty objects
|
||||
@ -158,8 +164,12 @@ parseOnConflict tn updFiltrM val = withPathK "on_conflict" $
|
||||
_ -> do
|
||||
UpdPermForIns _ updFiltr preSet <- onNothing updFiltrM $ throw500
|
||||
"cannot update columns since update permission is not defined"
|
||||
return $ RI.CP1Update constraint updCols preSet $
|
||||
toSQLBoolExp (S.mkQual tn) updFiltr
|
||||
preSetRes <- mapM (convPartialSQLExp sessVarFromCurrentSetting) preSet
|
||||
updFltrRes <- traverseAnnBoolExp
|
||||
(convPartialSQLExp sessVarFromCurrentSetting)
|
||||
updFiltr
|
||||
return $ RI.CP1Update constraint updCols preSetRes $
|
||||
toSQLBoolExp (S.mkQual tn) updFltrRes
|
||||
|
||||
where
|
||||
getUpdCols o = do
|
||||
@ -465,23 +475,31 @@ prefixErrPath fld =
|
||||
withPathK "selectionSet" . fieldAsPath fld . withPathK "args"
|
||||
|
||||
convertInsert
|
||||
:: RoleName
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r, Has InsCtxMap r
|
||||
)
|
||||
=> RoleName
|
||||
-> QualifiedTable -- table
|
||||
-> Field -- the mutation field
|
||||
-> Convert RespTx
|
||||
-> m RespTx
|
||||
convertInsert role tn fld = prefixErrPath fld $ do
|
||||
mutFlds <- convertMutResp (_fType fld) $ _fSelSet fld
|
||||
mutFldsUnres <- convertMutResp (_fType fld) $ _fSelSet fld
|
||||
mutFldsRes <- RR.traverseMutFlds resolveValTxt mutFldsUnres
|
||||
annVals <- withArg arguments "objects" asArray
|
||||
-- if insert input objects is empty array then
|
||||
-- do not perform insert and return mutation response
|
||||
bool (withNonEmptyObjs annVals mutFlds) (withEmptyObjs mutFlds) $ null annVals
|
||||
bool (withNonEmptyObjs annVals mutFldsRes)
|
||||
(withEmptyObjs mutFldsRes) $ null annVals
|
||||
where
|
||||
withNonEmptyObjs annVals mutFlds = do
|
||||
InsCtx vn tableCols defValMap relInfoMap updPerm <- getInsCtx tn
|
||||
annObjs <- mapM asObject annVals
|
||||
annInsObjs <- forM annObjs $ mkAnnInsObj relInfoMap
|
||||
conflictClauseM <- forM onConflictM $ parseOnConflict tn updPerm
|
||||
let multiObjIns = AnnIns annInsObjs conflictClauseM vn tableCols defValMap
|
||||
defValMapRes <- mapM (convPartialSQLExp sessVarFromCurrentSetting)
|
||||
defValMap
|
||||
let multiObjIns = AnnIns annInsObjs conflictClauseM
|
||||
vn tableCols defValMapRes
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ prefixErrPath fld $ insertMultipleObjects strfyNum role tn
|
||||
multiObjIns [] mutFlds "objects"
|
||||
@ -498,7 +516,8 @@ getInsCtx tn = do
|
||||
ctxMap <- asks getter
|
||||
insCtx <- onNothing (Map.lookup tn ctxMap) $
|
||||
throw500 $ "table " <> tn <<> " not found"
|
||||
let defValMap = S.mkColDefValMap $ map pgiName $ icAllCols insCtx
|
||||
let defValMap = fmap PSESQLExp $ S.mkColDefValMap $ map pgiName $
|
||||
icAllCols insCtx
|
||||
setCols = icSet insCtx
|
||||
return $ insCtx {icSet = Map.union setCols defValMap}
|
||||
|
||||
|
@ -1,165 +0,0 @@
|
||||
module Hasura.GraphQL.Resolve.LiveQuery
|
||||
( LiveQuery(..)
|
||||
, LiveQueryMap
|
||||
, newLiveQueryMap
|
||||
, addLiveQuery
|
||||
, TxRunner
|
||||
, removeLiveQuery
|
||||
) where
|
||||
|
||||
import qualified Control.Concurrent.Async as A
|
||||
import qualified Control.Concurrent.STM as STM
|
||||
import qualified ListT
|
||||
import qualified StmContainers.Map as STMMap
|
||||
|
||||
import Control.Concurrent (threadDelay)
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Resolve.Context (LazyRespTx)
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types
|
||||
|
||||
data LiveQuery
|
||||
= LiveQuery
|
||||
{ _lqUser :: !UserInfo
|
||||
, _lqRequest :: !GraphQLRequest
|
||||
} deriving (Show, Eq, Generic)
|
||||
|
||||
instance Hashable LiveQuery
|
||||
|
||||
type OnChange k = GQResp -> IO ()
|
||||
|
||||
data LQHandler k
|
||||
= LQHandler
|
||||
-- the tx to be executed
|
||||
{ _lqhRespTx :: !LazyRespTx
|
||||
-- previous result
|
||||
, _lqhPrevRes :: !(STM.TVar (Maybe GQResp))
|
||||
-- the actions that have been run previously
|
||||
-- we run these if the response changes
|
||||
, _lqhCurOps :: !(STMMap.Map k (OnChange k))
|
||||
-- we run these operations regardless
|
||||
-- and then merge them with current operations
|
||||
, _lqhNewOps :: !(STMMap.Map k (OnChange k))
|
||||
}
|
||||
|
||||
type ThreadTM = STM.TMVar (A.Async ())
|
||||
type LiveQueryMap k = STMMap.Map LiveQuery (LQHandler k, ThreadTM)
|
||||
|
||||
newLiveQueryMap :: STM.STM (LiveQueryMap k)
|
||||
newLiveQueryMap = STMMap.new
|
||||
|
||||
type TxRunner = LazyRespTx -> IO (Either QErr EncJSON)
|
||||
|
||||
removeLiveQuery
|
||||
:: (Eq k, Hashable k)
|
||||
=> LiveQueryMap k
|
||||
-- the query and the associated operation
|
||||
-> LiveQuery
|
||||
-> k
|
||||
-> IO ()
|
||||
removeLiveQuery lqMap liveQ k = do
|
||||
|
||||
-- clean the handler's state
|
||||
threadRefM <- STM.atomically $ do
|
||||
lqHandlerM <- STMMap.lookup liveQ lqMap
|
||||
maybe (return Nothing) cleanLQHandler lqHandlerM
|
||||
|
||||
-- cancel the polling thread
|
||||
onJust threadRefM A.cancel
|
||||
|
||||
where
|
||||
cleanLQHandler (handler, threadRef) = do
|
||||
let curOps = _lqhCurOps handler
|
||||
newOps = _lqhNewOps handler
|
||||
STMMap.delete k curOps
|
||||
STMMap.delete k newOps
|
||||
cancelPollThread <- (&&)
|
||||
<$> STMMap.null curOps
|
||||
<*> STMMap.null newOps
|
||||
-- if this happens to be the last operation, take the
|
||||
-- ref for the polling thread to cancel it
|
||||
if cancelPollThread then do
|
||||
STMMap.delete liveQ lqMap
|
||||
Just <$> STM.takeTMVar threadRef
|
||||
else return Nothing
|
||||
|
||||
addLiveQuery
|
||||
:: (Eq k, Hashable k)
|
||||
=> TxRunner
|
||||
-> LiveQueryMap k
|
||||
-- the query
|
||||
-> LiveQuery
|
||||
-- the transaction associated with this query
|
||||
-> LazyRespTx
|
||||
-- a unique operation id
|
||||
-> k
|
||||
-- the action to be executed when result changes
|
||||
-> OnChange k
|
||||
-> IO ()
|
||||
addLiveQuery txRunner lqMap liveQ respTx k onResultAction= do
|
||||
|
||||
-- a handler is returned only when it is newly created
|
||||
handlerM <- STM.atomically $ do
|
||||
lqHandlerM <- STMMap.lookup liveQ lqMap
|
||||
maybe newHandler addToExistingHandler lqHandlerM
|
||||
|
||||
-- we can then attach a polling thread if it is new
|
||||
-- the livequery can only be cancelled after putTMVar
|
||||
onJust handlerM $ \(handler, pollerThreadTM) -> do
|
||||
threadRef <- A.async $ forever $ do
|
||||
pollQuery txRunner handler
|
||||
threadDelay $ 1 * 1000 * 1000
|
||||
STM.atomically $ STM.putTMVar pollerThreadTM threadRef
|
||||
|
||||
where
|
||||
|
||||
addToExistingHandler (handler, _) = do
|
||||
STMMap.insert onResultAction k $ _lqhNewOps handler
|
||||
return Nothing
|
||||
|
||||
newHandler = do
|
||||
handler <- LQHandler
|
||||
<$> return respTx
|
||||
<*> STM.newTVar Nothing
|
||||
<*> STMMap.new
|
||||
<*> STMMap.new
|
||||
STMMap.insert onResultAction k $ _lqhNewOps handler
|
||||
asyncRefTM <- STM.newEmptyTMVar
|
||||
STMMap.insert (handler, asyncRefTM) liveQ lqMap
|
||||
return $ Just (handler, asyncRefTM)
|
||||
|
||||
pollQuery
|
||||
:: (Eq k, Hashable k)
|
||||
=> TxRunner
|
||||
-> LQHandler k
|
||||
-> IO ()
|
||||
pollQuery runTx (LQHandler respTx respTV curOpsTV newOpsTV) = do
|
||||
|
||||
res <- runTx respTx
|
||||
|
||||
let resp = case res of
|
||||
Left e -> GQExecError [encodeGQErr False e]
|
||||
Right bs -> GQSuccess $ encJToLBS bs
|
||||
|
||||
-- extract the current and new operations
|
||||
(curOps, newOps) <- STM.atomically $ do
|
||||
curOpsL <- ListT.toList $ STMMap.listT curOpsTV
|
||||
newOpsL <- ListT.toList $ STMMap.listT newOpsTV
|
||||
forM_ newOpsL $ \(k, action) -> STMMap.insert action k curOpsTV
|
||||
STMMap.reset newOpsTV
|
||||
return (curOpsL, newOpsL)
|
||||
|
||||
runOperations resp newOps
|
||||
|
||||
-- write to the current websockets if needed
|
||||
prevRespM <- STM.readTVarIO respTV
|
||||
when (isExecError resp || Just resp /= prevRespM) $ do
|
||||
runOperations resp curOps
|
||||
STM.atomically $ STM.writeTVar respTV $ Just resp
|
||||
|
||||
where
|
||||
runOperation resp action = action resp
|
||||
runOperations resp =
|
||||
void . A.mapConcurrently (runOperation resp . snd)
|
@ -19,41 +19,57 @@ import qualified Hasura.RQL.DML.Returning as RR
|
||||
import qualified Hasura.RQL.DML.Update as RU
|
||||
|
||||
import qualified Hasura.SQL.DML as S
|
||||
import qualified Hasura.RQL.DML.Select as RS
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Context
|
||||
import Hasura.GraphQL.Resolve.BoolExp
|
||||
import Hasura.GraphQL.Resolve.Context
|
||||
import Hasura.GraphQL.Resolve.InputValue
|
||||
import Hasura.GraphQL.Resolve.Select (fromSelSet, withSelSet)
|
||||
import Hasura.GraphQL.Resolve.Select (fromSelSet)
|
||||
import Hasura.GraphQL.Validate.Field
|
||||
import Hasura.GraphQL.Validate.Types
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
-- withPrepFn
|
||||
-- :: (MonadReader r m)
|
||||
-- => PrepFn m -> ReaderT (r, PrepFn m) m a -> m a
|
||||
-- withPrepFn fn m = do
|
||||
-- r <- ask
|
||||
-- runReaderT m (r, fn)
|
||||
|
||||
convertMutResp
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> G.NamedType -> SelSet -> m RR.MutFlds
|
||||
=> G.NamedType -> SelSet -> m (RR.MutFldsG UnresolvedVal)
|
||||
convertMutResp ty selSet =
|
||||
withSelSet selSet $ \fld -> case _fName fld of
|
||||
"__typename" -> return $ RR.MExp $ G.unName $ G.unNamedType ty
|
||||
"affected_rows" -> return RR.MCount
|
||||
"returning" -> fmap RR.MRet $
|
||||
fromSelSet txtConverter (_fType fld) $ _fSelSet fld
|
||||
"returning" -> do
|
||||
annFlds <- fromSelSet (_fType fld) $ _fSelSet fld
|
||||
annFldsResolved <- traverse
|
||||
(traverse (RS.traverseAnnFld convertUnresolvedVal)) annFlds
|
||||
return $ RR.MRet annFldsResolved
|
||||
G.Name t -> throw500 $ "unexpected field in mutation resp : " <> t
|
||||
where
|
||||
convertUnresolvedVal = \case
|
||||
UVPG annPGVal -> UVSQL <$> txtConverter annPGVal
|
||||
UVSessVar colTy sessVar -> pure $ UVSessVar colTy sessVar
|
||||
UVSQL sqlExp -> pure $ UVSQL sqlExp
|
||||
|
||||
convertRowObj
|
||||
:: (MonadError QErr m, MonadState PrepArgs m)
|
||||
:: (MonadError QErr m)
|
||||
=> AnnInpVal
|
||||
-> m [(PGCol, S.SQLExp)]
|
||||
-> m [(PGCol, UnresolvedVal)]
|
||||
convertRowObj val =
|
||||
flip withObject val $ \_ obj ->
|
||||
forM (OMap.toList obj) $ \(k, v) -> do
|
||||
prepExpM <- asPGColValM v >>= mapM prepare
|
||||
let prepExp = fromMaybe (S.SEUnsafe "NULL") prepExpM
|
||||
prepExpM <- fmap UVPG <$> asPGColValM v
|
||||
let prepExp = fromMaybe (UVSQL $ S.SEUnsafe "NULL") prepExpM
|
||||
return (PGCol $ G.unName k, prepExp)
|
||||
|
||||
type ApplySQLOp = (PGCol, S.SQLExp) -> S.SQLExp
|
||||
@ -72,18 +88,19 @@ lhsExpOp op annTy (col, e) =
|
||||
|
||||
convObjWithOp
|
||||
:: (MonadError QErr m)
|
||||
=> ApplySQLOp -> AnnInpVal -> m [(PGCol, S.SQLExp)]
|
||||
=> ApplySQLOp -> AnnInpVal -> m [(PGCol, UnresolvedVal)]
|
||||
convObjWithOp opFn val =
|
||||
flip withObject val $ \_ obj -> forM (OMap.toList obj) $ \(k, v) -> do
|
||||
colVal <- _apvValue <$> asPGColVal v
|
||||
let pgCol = PGCol $ G.unName k
|
||||
-- TODO: why are we using txtEncoder here?
|
||||
encVal = txtEncoder colVal
|
||||
sqlExp = opFn (pgCol, encVal)
|
||||
return (pgCol, sqlExp)
|
||||
return (pgCol, UVSQL sqlExp)
|
||||
|
||||
convDeleteAtPathObj
|
||||
:: (MonadError QErr m)
|
||||
=> AnnInpVal -> m [(PGCol, S.SQLExp)]
|
||||
=> AnnInpVal -> m [(PGCol, UnresolvedVal)]
|
||||
convDeleteAtPathObj val =
|
||||
flip withObject val $ \_ obj -> forM (OMap.toList obj) $ \(k, v) -> do
|
||||
vals <- flip withArray v $ \_ annVals -> mapM asPGColVal annVals
|
||||
@ -92,22 +109,21 @@ convDeleteAtPathObj val =
|
||||
annEncVal = S.SETyAnn (S.SEArray valExps) S.textArrType
|
||||
sqlExp = S.SEOpApp S.jsonbDeleteAtPathOp
|
||||
[S.SEIden $ toIden pgCol, annEncVal]
|
||||
return (pgCol, sqlExp)
|
||||
return (pgCol, UVSQL sqlExp)
|
||||
|
||||
convertUpdateP1
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
, MonadState PrepArgs m
|
||||
)
|
||||
=> UpdOpCtx -- the update context
|
||||
-> Field -- the mutation field
|
||||
-> m RU.UpdateQueryP1
|
||||
-> m (RU.AnnUpdG UnresolvedVal)
|
||||
convertUpdateP1 opCtx fld = do
|
||||
-- a set expression is same as a row object
|
||||
setExpM <- withArgM args "_set" convertRowObj
|
||||
-- where bool expression to filter column
|
||||
whereExp <- withArg args "where" (parseBoolExp prepare)
|
||||
whereExp <- withArg args "where" parseBoolExp
|
||||
-- increment operator on integer columns
|
||||
incExpM <- withArgM args "_inc" $
|
||||
convObjWithOp $ rhsExpOp S.incOp S.intType
|
||||
@ -125,54 +141,74 @@ convertUpdateP1 opCtx fld = do
|
||||
convObjWithOp $ rhsExpOp S.jsonbDeleteOp S.intType
|
||||
-- delete at path in jsonb value
|
||||
deleteAtPathExpM <- withArgM args "_delete_at_path" convDeleteAtPathObj
|
||||
mutFlds <- convertMutResp (_fType fld) $ _fSelSet fld
|
||||
mutFlds <- convertMutResp (_fType fld) $ _fSelSet fld
|
||||
|
||||
let resolvedPreSetItems =
|
||||
Map.toList $ fmap partialSQLExpToUnresolvedVal preSetCols
|
||||
|
||||
let updExpsM = [ setExpM, incExpM, appendExpM, prependExpM
|
||||
, deleteKeyExpM, deleteElemExpM, deleteAtPathExpM
|
||||
]
|
||||
setItems = preSetItems ++ concat (catMaybes updExpsM)
|
||||
setItems = resolvedPreSetItems ++ concat (catMaybes updExpsM)
|
||||
|
||||
-- atleast one of update operators is expected
|
||||
-- or preSetItems shouldn't be empty
|
||||
-- this is not equivalent to (null setItems)
|
||||
unless (any isJust updExpsM || not (null preSetItems)) $ throwVE $
|
||||
unless (any isJust updExpsM || not (null resolvedPreSetItems)) $ throwVE $
|
||||
"atleast any one of _set, _inc, _append, _prepend, "
|
||||
<> "_delete_key, _delete_elem and "
|
||||
<> "_delete_at_path operator is expected"
|
||||
|
||||
return $ RU.UpdateQueryP1 tn setItems (filterExp, whereExp) mutFlds allCols
|
||||
let unresolvedPermFltr = fmapAnnBoolExp partialSQLExpToUnresolvedVal filterExp
|
||||
|
||||
return $ RU.AnnUpd tn setItems
|
||||
(unresolvedPermFltr, whereExp) mutFlds allCols
|
||||
where
|
||||
UpdOpCtx tn _ filterExp preSetCols allCols = opCtx
|
||||
args = _fArguments fld
|
||||
preSetItems = Map.toList preSetCols
|
||||
|
||||
convertUpdate
|
||||
:: UpdOpCtx -- the update context
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> UpdOpCtx -- the update context
|
||||
-> Field -- the mutation field
|
||||
-> Convert RespTx
|
||||
-> m RespTx
|
||||
convertUpdate opCtx fld = do
|
||||
(p1, prepArgs) <- withPrepArgs $ convertUpdateP1 opCtx fld
|
||||
annUpdUnresolved <- convertUpdateP1 opCtx fld
|
||||
(annUpdResolved, prepArgs) <- withPrepArgs $ RU.traverseAnnUpd
|
||||
resolveValPrep annUpdUnresolved
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
let whenNonEmptyItems = return $ RU.updateQueryToTx strfyNum (p1, prepArgs)
|
||||
let whenNonEmptyItems = return $ RU.updateQueryToTx strfyNum
|
||||
(annUpdResolved, prepArgs)
|
||||
whenEmptyItems = return $ return $
|
||||
buildEmptyMutResp $ RU.uqp1MutFlds p1
|
||||
buildEmptyMutResp $ RU.uqp1MutFlds annUpdResolved
|
||||
-- if there are not set items then do not perform
|
||||
-- update and return empty mutation response
|
||||
bool whenNonEmptyItems whenEmptyItems $ null $ RU.uqp1SetExps p1
|
||||
bool whenNonEmptyItems whenEmptyItems $ null $ RU.uqp1SetExps annUpdResolved
|
||||
|
||||
convertDelete
|
||||
:: DelOpCtx -- the delete context
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> DelOpCtx -- the delete context
|
||||
-> Field -- the mutation field
|
||||
-> Convert RespTx
|
||||
-> m RespTx
|
||||
convertDelete opCtx fld = do
|
||||
(p1, prepArgs) <- p1m
|
||||
whereExp <- withArg (_fArguments fld) "where" parseBoolExp
|
||||
mutFlds <- convertMutResp (_fType fld) $ _fSelSet fld
|
||||
let unresolvedPermFltr =
|
||||
fmapAnnBoolExp partialSQLExpToUnresolvedVal filterExp
|
||||
annDelUnresolved = RD.AnnDel tn (unresolvedPermFltr, whereExp)
|
||||
mutFlds allCols
|
||||
(annDelResolved, prepArgs) <- withPrepArgs $ RD.traverseAnnDel
|
||||
resolveValPrep annDelUnresolved
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ RD.deleteQueryToTx strfyNum (p1, prepArgs)
|
||||
return $ RD.deleteQueryToTx strfyNum (annDelResolved, prepArgs)
|
||||
where
|
||||
DelOpCtx tn _ filterExp allCols = opCtx
|
||||
p1m = withPrepArgs $ do
|
||||
whereExp <- withArg (_fArguments fld) "where" (parseBoolExp prepare)
|
||||
mutFlds <- convertMutResp (_fType fld) $ _fSelSet fld
|
||||
return $ RD.DeleteQueryP1 tn (filterExp, whereExp) mutFlds allCols
|
||||
|
||||
-- | build mutation response for empty objects
|
||||
buildEmptyMutResp :: RR.MutFlds -> EncJSON
|
||||
|
@ -2,15 +2,15 @@ module Hasura.GraphQL.Resolve.Select
|
||||
( convertSelect
|
||||
, convertSelectByPKey
|
||||
, convertAggSelect
|
||||
, convertFuncQuery
|
||||
, convertFuncQuerySimple
|
||||
, convertFuncQueryAgg
|
||||
, parseColumns
|
||||
, withSelSet
|
||||
, fromSelSet
|
||||
, fieldAsPath
|
||||
, fromField
|
||||
, fromFieldByPKey
|
||||
, fromAggField
|
||||
, fromFuncQueryField
|
||||
, QueryRootFldAST(..)
|
||||
, traverseQueryRootFldAST
|
||||
, QueryRootFldUnresolved
|
||||
, QueryRootFldResolved
|
||||
, toPGQuery
|
||||
) where
|
||||
|
||||
import Control.Arrow (first)
|
||||
@ -24,6 +24,7 @@ import qualified Data.List.NonEmpty as NE
|
||||
import qualified Data.Text as T
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Hasura.RQL.DML.Select as RS
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
@ -39,12 +40,6 @@ import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value
|
||||
|
||||
withSelSet :: (Monad m) => SelSet -> (Field -> m a) -> m [(Text, a)]
|
||||
withSelSet selSet f =
|
||||
forM (toList selSet) $ \fld -> do
|
||||
res <- f fld
|
||||
return (G.unName $ G.unAlias $ _fAlias fld, res)
|
||||
|
||||
jsonPathToColExp :: (MonadError QErr m) => T.Text -> m S.SQLExp
|
||||
jsonPathToColExp t = case parseJSONPath t of
|
||||
Left s -> throw400 ParseFailed $ T.pack $ "parse json path error: " ++ s
|
||||
@ -60,12 +55,14 @@ argsToColOp args = maybe (return Nothing) toOp $ Map.lookup "path" args
|
||||
toJsonPathExp = fmap (RS.ColOp S.jsonbPathOp) . jsonPathToColExp
|
||||
toOp v = asPGColTextM v >>= mapM toJsonPathExp
|
||||
|
||||
type AnnFlds = RS.AnnFldsG UnresolvedVal
|
||||
|
||||
fromSelSet
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> PrepFn m -> G.NamedType -> SelSet -> m RS.AnnFlds
|
||||
fromSelSet f fldTy flds =
|
||||
=> G.NamedType -> SelSet -> m AnnFlds
|
||||
fromSelSet fldTy flds =
|
||||
forM (toList flds) $ \fld -> do
|
||||
let fldName = _fName fld
|
||||
let rqlFldName = FieldName $ G.unName $ G.unAlias $ _fAlias fld
|
||||
@ -75,51 +72,51 @@ fromSelSet f fldTy flds =
|
||||
fldInfo <- getFldInfo fldTy fldName
|
||||
case fldInfo of
|
||||
Left colInfo ->
|
||||
(RS.FCol colInfo) <$> (argsToColOp $ _fArguments fld)
|
||||
-- let jsonCol = return $ RS.FCol $ colInfo { pgiName = PGCol $ T.pack "metadata->'name'" }
|
||||
RS.FCol colInfo <$> argsToColOp (_fArguments fld)
|
||||
Right (relInfo, isAgg, tableFilter, tableLimit) -> do
|
||||
let relTN = riRTable relInfo
|
||||
colMapping = riMapping relInfo
|
||||
rn = riName relInfo
|
||||
if isAgg then do
|
||||
aggSel <- fromAggField f relTN tableFilter tableLimit fld
|
||||
aggSel <- fromAggField relTN tableFilter tableLimit fld
|
||||
return $ RS.FArr $ RS.ASAgg $ RS.AnnRelG rn colMapping aggSel
|
||||
else do
|
||||
annSel <- fromField f relTN tableFilter tableLimit fld
|
||||
annSel <- fromField relTN tableFilter tableLimit fld
|
||||
let annRel = RS.AnnRelG rn colMapping annSel
|
||||
return $ case riType relInfo of
|
||||
ObjRel -> RS.FObj annRel
|
||||
ArrRel -> RS.FArr $ RS.ASSimple annRel
|
||||
|
||||
type TableAggFlds = RS.TableAggFldsG UnresolvedVal
|
||||
|
||||
fromAggSelSet
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> PrepFn m -> G.NamedType -> SelSet -> m RS.TableAggFlds
|
||||
fromAggSelSet fn fldTy selSet = fmap toFields $
|
||||
=> G.NamedType -> SelSet -> m TableAggFlds
|
||||
fromAggSelSet fldTy selSet = fmap toFields $
|
||||
withSelSet selSet $ \f -> do
|
||||
let fTy = _fType f
|
||||
fSelSet = _fSelSet f
|
||||
case _fName f of
|
||||
"__typename" -> return $ RS.TAFExp $ G.unName $ G.unNamedType fldTy
|
||||
"aggregate" -> RS.TAFAgg <$> convertAggFld fTy fSelSet
|
||||
"nodes" -> RS.TAFNodes <$> fromSelSet fn fTy fSelSet
|
||||
"nodes" -> RS.TAFNodes <$> fromSelSet fTy fSelSet
|
||||
G.Name t -> throw500 $ "unexpected field in _agg node: " <> t
|
||||
|
||||
fieldAsPath :: (MonadError QErr m) => Field -> m a -> m a
|
||||
fieldAsPath = nameAsPath . _fName
|
||||
type TableArgs = RS.TableArgsG UnresolvedVal
|
||||
|
||||
parseTableArgs
|
||||
:: ( MonadError QErr m, MonadReader r m
|
||||
, Has FieldMap r, Has OrdByCtx r
|
||||
)
|
||||
=> PrepFn m -> ArgsMap -> m RS.TableArgs
|
||||
parseTableArgs f args = do
|
||||
whereExpM <- withArgM args "where" $ parseBoolExp f
|
||||
=> ArgsMap -> m TableArgs
|
||||
parseTableArgs args = do
|
||||
whereExpM <- withArgM args "where" parseBoolExp
|
||||
ordByExpML <- withArgM args "order_by" parseOrderBy
|
||||
let ordByExpM = NE.nonEmpty =<< ordByExpML
|
||||
limitExpM <- withArgM args "limit" parseLimit
|
||||
offsetExpM <- withArgM args "offset" $ asPGColVal >=> f
|
||||
offsetExpM <- withArgM args "offset" $ asPGColVal >=> txtConverter
|
||||
distOnColsML <- withArgM args "distinct_on" parseColumns
|
||||
let distOnColsM = NE.nonEmpty =<< distOnColsML
|
||||
mapM_ (validateDistOn ordByExpM) distOnColsM
|
||||
@ -139,17 +136,20 @@ parseTableArgs f args = do
|
||||
unless isValid $ throwVE
|
||||
"\"distinct_on\" columns must match initial \"order_by\" columns"
|
||||
|
||||
type AnnSimpleSelect = RS.AnnSimpleSelG UnresolvedVal
|
||||
|
||||
fromField
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> PrepFn m -> QualifiedTable -> AnnBoolExpSQL
|
||||
-> Maybe Int -> Field -> m RS.AnnSel
|
||||
fromField f tn permFilter permLimitM fld = fieldAsPath fld $ do
|
||||
tableArgs <- parseTableArgs f args
|
||||
annFlds <- fromSelSet f (_fType fld) $ _fSelSet fld
|
||||
=> QualifiedTable -> AnnBoolExpPartialSQL
|
||||
-> Maybe Int -> Field -> m AnnSimpleSelect
|
||||
fromField tn permFilter permLimitM fld = fieldAsPath fld $ do
|
||||
tableArgs <- parseTableArgs args
|
||||
annFlds <- fromSelSet (_fType fld) $ _fSelSet fld
|
||||
let unresolvedPermFltr = fmapAnnBoolExp partialSQLExpToUnresolvedVal permFilter
|
||||
let tabFrom = RS.TableFrom tn Nothing
|
||||
tabPerm = RS.TablePerm permFilter permLimitM
|
||||
tabPerm = RS.TablePerm unresolvedPermFltr permLimitM
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ RS.AnnSelG annFlds tabFrom tabPerm tableArgs strfyNum
|
||||
where
|
||||
@ -171,7 +171,7 @@ parseOrderBy
|
||||
, MonadReader r m
|
||||
, Has OrdByCtx r
|
||||
)
|
||||
=> AnnInpVal -> m [RS.AnnOrderByItem]
|
||||
=> AnnInpVal -> m [RS.AnnOrderByItemG UnresolvedVal]
|
||||
parseOrderBy = fmap concat . withArray f
|
||||
where
|
||||
f _ = mapM (withObject (getAnnObItems id))
|
||||
@ -181,10 +181,10 @@ getAnnObItems
|
||||
, MonadReader r m
|
||||
, Has OrdByCtx r
|
||||
)
|
||||
=> (RS.AnnObCol -> RS.AnnObCol)
|
||||
=> (RS.AnnObColG UnresolvedVal -> RS.AnnObColG UnresolvedVal)
|
||||
-> G.NamedType
|
||||
-> AnnGObject
|
||||
-> m [RS.AnnOrderByItem]
|
||||
-> m [RS.AnnOrderByItemG UnresolvedVal]
|
||||
getAnnObItems f nt obj = do
|
||||
ordByItemMap <- getOrdByItemMap nt
|
||||
fmap concat $ forM (OMap.toList obj) $ \(k, v) -> do
|
||||
@ -198,11 +198,13 @@ getAnnObItems f nt obj = do
|
||||
(ordTy, nullsOrd) <- parseOrderByEnum enumVal
|
||||
return [mkOrdByItemG ordTy aobCol nullsOrd]
|
||||
OBIRel ri fltr -> do
|
||||
let annObColFn = f . RS.AOCObj ri fltr
|
||||
let unresolvedFltr = fmapAnnBoolExp partialSQLExpToUnresolvedVal fltr
|
||||
let annObColFn = f . RS.AOCObj ri unresolvedFltr
|
||||
withObject (getAnnObItems annObColFn) v
|
||||
|
||||
OBIAgg ri fltr -> do
|
||||
let aobColFn = f . RS.AOCAgg ri fltr
|
||||
let unresolvedFltr = fmapAnnBoolExp partialSQLExpToUnresolvedVal fltr
|
||||
let aobColFn = f . RS.AOCAgg ri unresolvedFltr
|
||||
flip withObject v $ \_ o -> parseAggOrdBy aobColFn o
|
||||
|
||||
mkOrdByItemG :: S.OrderType -> a -> S.NullsOrder -> OrderByItemG a
|
||||
@ -211,9 +213,9 @@ mkOrdByItemG ordTy aobCol nullsOrd =
|
||||
|
||||
parseAggOrdBy
|
||||
:: (MonadError QErr m)
|
||||
=> (RS.AnnAggOrdBy -> RS.AnnObCol)
|
||||
=> (RS.AnnAggOrdBy -> RS.AnnObColG UnresolvedVal)
|
||||
-> AnnGObject
|
||||
-> m [RS.AnnOrderByItem]
|
||||
-> m [RS.AnnOrderByItemG UnresolvedVal]
|
||||
parseAggOrdBy f annObj =
|
||||
fmap concat <$> forM (OMap.toList annObj) $ \(op, obVal) ->
|
||||
case op of
|
||||
@ -256,6 +258,25 @@ parseLimit v = do
|
||||
where
|
||||
noIntErr = throwVE "expecting Integer value for \"limit\""
|
||||
|
||||
type AnnSimpleSel = RS.AnnSimpleSelG UnresolvedVal
|
||||
|
||||
type PGColValMap = Map.HashMap G.Name AnnInpVal
|
||||
|
||||
pgColValToBoolExp
|
||||
:: (MonadError QErr m)
|
||||
=> PGColArgMap -> PGColValMap -> m AnnBoolExpUnresolved
|
||||
pgColValToBoolExp colArgMap colValMap = do
|
||||
colExps <- forM colVals $ \(name, val) ->
|
||||
BoolFld <$> do
|
||||
opExp <- AEQ True . UVPG <$> asPGColVal val
|
||||
colInfo <- onNothing (Map.lookup name colArgMap) $
|
||||
throw500 $ "column name " <> showName name
|
||||
<> " not found in column arguments map"
|
||||
return $ AVCol colInfo [opExp]
|
||||
return $ BoolAnd colExps
|
||||
where
|
||||
colVals = Map.toList colValMap
|
||||
|
||||
fromFieldByPKey
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
@ -263,13 +284,15 @@ fromFieldByPKey
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
)
|
||||
=> PrepFn m -> QualifiedTable -> PGColArgMap
|
||||
-> AnnBoolExpSQL -> Field -> m RS.AnnSel
|
||||
fromFieldByPKey f tn colArgMap permFilter fld = fieldAsPath fld $ do
|
||||
boolExp <- pgColValToBoolExp f colArgMap $ _fArguments fld
|
||||
annFlds <- fromSelSet f fldTy $ _fSelSet fld
|
||||
=> QualifiedTable -> PGColArgMap
|
||||
-> AnnBoolExpPartialSQL -> Field -> m AnnSimpleSel
|
||||
fromFieldByPKey tn colArgMap permFilter fld = fieldAsPath fld $ do
|
||||
boolExp <- pgColValToBoolExp colArgMap $ _fArguments fld
|
||||
annFlds <- fromSelSet fldTy $ _fSelSet fld
|
||||
let tabFrom = RS.TableFrom tn Nothing
|
||||
tabPerm = RS.TablePerm permFilter Nothing
|
||||
unresolvedPermFltr = fmapAnnBoolExp partialSQLExpToUnresolvedVal
|
||||
permFilter
|
||||
tabPerm = RS.TablePerm unresolvedPermFltr Nothing
|
||||
tabArgs = RS.noTableArgs { RS._taWhere = Just boolExp}
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ RS.AnnSelG annFlds tabFrom tabPerm tabArgs strfyNum
|
||||
@ -277,22 +300,24 @@ fromFieldByPKey f tn colArgMap permFilter fld = fieldAsPath fld $ do
|
||||
fldTy = _fType fld
|
||||
|
||||
convertSelect
|
||||
:: SelOpCtx -> Field -> Convert RespTx
|
||||
convertSelect opCtx fld = do
|
||||
(selData, prepArgs) <-
|
||||
withPathK "selectionSet" $ withPrepArgs $
|
||||
fromField prepare qt permFilter permLimit fld
|
||||
return $ RS.selectP2 False (selData, prepArgs)
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> SelOpCtx -> Field -> m QueryRootFldUnresolved
|
||||
convertSelect opCtx fld =
|
||||
withPathK "selectionSet" $ fmap QRFSimple $
|
||||
fromField qt permFilter permLimit fld
|
||||
where
|
||||
SelOpCtx qt _ permFilter permLimit = opCtx
|
||||
|
||||
convertSelectByPKey
|
||||
:: SelPkOpCtx -> Field -> Convert RespTx
|
||||
convertSelectByPKey opCtx fld = do
|
||||
(selData, prepArgs) <-
|
||||
withPathK "selectionSet" $ withPrepArgs $
|
||||
fromFieldByPKey prepare qt colArgMap permFilter fld
|
||||
return $ RS.selectP2 True (selData, prepArgs)
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> SelPkOpCtx -> Field -> m QueryRootFldUnresolved
|
||||
convertSelectByPKey opCtx fld =
|
||||
withPathK "selectionSet" $ fmap QRFPk $
|
||||
fromFieldByPKey qt colArgMap permFilter fld
|
||||
where
|
||||
SelPkOpCtx qt _ permFilter colArgMap = opCtx
|
||||
|
||||
@ -349,71 +374,115 @@ convertAggFld ty selSet = fmap toFields $
|
||||
throwInvalidFld (G.Name t) =
|
||||
throw500 $ "unexpected field in _aggregate node: " <> t
|
||||
|
||||
type AnnAggSel = RS.AnnAggSelG UnresolvedVal
|
||||
|
||||
fromAggField
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> PrepFn m -> QualifiedTable -> AnnBoolExpSQL
|
||||
-> Maybe Int -> Field -> m RS.AnnAggSel
|
||||
fromAggField f tn permFilter permLimit fld = fieldAsPath fld $ do
|
||||
tableArgs <- parseTableArgs f args
|
||||
aggSelFlds <- fromAggSelSet f (_fType fld) (_fSelSet fld)
|
||||
=> QualifiedTable -> AnnBoolExpPartialSQL
|
||||
-> Maybe Int -> Field -> m AnnAggSel
|
||||
fromAggField tn permFilter permLimit fld = fieldAsPath fld $ do
|
||||
tableArgs <- parseTableArgs args
|
||||
aggSelFlds <- fromAggSelSet (_fType fld) (_fSelSet fld)
|
||||
let unresolvedPermFltr =
|
||||
fmapAnnBoolExp partialSQLExpToUnresolvedVal permFilter
|
||||
let tabFrom = RS.TableFrom tn Nothing
|
||||
tabPerm = RS.TablePerm permFilter permLimit
|
||||
tabPerm = RS.TablePerm unresolvedPermFltr permLimit
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ RS.AnnSelG aggSelFlds tabFrom tabPerm tableArgs strfyNum
|
||||
where
|
||||
args = _fArguments fld
|
||||
|
||||
convertAggSelect :: SelOpCtx -> Field -> Convert RespTx
|
||||
convertAggSelect opCtx fld = do
|
||||
(selData, prepArgs) <-
|
||||
withPathK "selectionSet" $ withPrepArgs $
|
||||
fromAggField prepare qt permFilter permLimit fld
|
||||
return $ RS.selectAggP2 (selData, prepArgs)
|
||||
convertAggSelect
|
||||
:: ( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> SelOpCtx -> Field -> m QueryRootFldUnresolved
|
||||
convertAggSelect opCtx fld =
|
||||
withPathK "selectionSet" $ fmap QRFAgg $
|
||||
fromAggField qt permFilter permLimit fld
|
||||
-- return $ RS.selectAggQuerySQL selData
|
||||
where
|
||||
SelOpCtx qt _ permFilter permLimit = opCtx
|
||||
|
||||
fromFuncQueryField
|
||||
::( MonadError QErr m, MonadReader r m, Has FieldMap r
|
||||
, Has OrdByCtx r, Has SQLGenCtx r
|
||||
)
|
||||
=> PrepFn m -> QualifiedFunction -> FuncArgSeq -> Bool -> Field
|
||||
-> m (RS.TableArgs, Either RS.TableAggFlds RS.AnnFlds, S.FromItem)
|
||||
fromFuncQueryField f qf argSeq isAgg fld = fieldAsPath fld $ do
|
||||
funcArgsM <- withArgM args "args" $ parseFunctionArgs f argSeq
|
||||
let funcArgs = fromMaybe [] funcArgsM
|
||||
funcFrmItem = S.mkFuncFromItem qf funcArgs
|
||||
tableArgs <- parseTableArgs f args
|
||||
eSelFlds <- bool nonAggSel aggSel isAgg
|
||||
return (tableArgs, eSelFlds, funcFrmItem)
|
||||
where
|
||||
args = _fArguments fld
|
||||
|
||||
nonAggSel = Right <$>
|
||||
fromSelSet f (_fType fld) (_fSelSet fld)
|
||||
aggSel = Left <$>
|
||||
fromAggSelSet f (_fType fld) (_fSelSet fld)
|
||||
|
||||
parseFunctionArgs
|
||||
::(MonadError QErr m)
|
||||
=> PrepFn m -> FuncArgSeq -> AnnInpVal -> m [S.SQLExp]
|
||||
parseFunctionArgs fn argSeq val =
|
||||
::( MonadError QErr m)
|
||||
=> FuncArgSeq -> AnnInpVal -> m [AnnPGVal]
|
||||
parseFunctionArgs argSeq val =
|
||||
flip withObject val $ \nTy obj ->
|
||||
fmap toList $ forM argSeq $ \(FuncArgItem argName) -> do
|
||||
argVal <- onNothing (OMap.lookup argName obj) $ throw500 $
|
||||
"argument " <> showName argName <> " required in input type "
|
||||
<> showNamedTy nTy
|
||||
fn =<< asPGColVal argVal
|
||||
asPGColVal argVal
|
||||
|
||||
convertFuncQuery
|
||||
:: FuncQOpCtx -> Bool -> Field -> Convert RespTx
|
||||
convertFuncQuery funcOpCtx isAgg fld = do
|
||||
((tableArgs, sel, frmItem), prepArgs) <-
|
||||
withPathK "selectionSet" $ withPrepArgs $
|
||||
fromFuncQueryField prepare qf argSeq isAgg fld
|
||||
let tabPerm = RS.TablePerm permFilter permLimit
|
||||
strfyNum <- stringifyNum <$> asks getter
|
||||
return $ RS.funcQueryTx frmItem qf qt tabPerm tableArgs strfyNum (sel, prepArgs)
|
||||
fromFuncQueryField
|
||||
:: (MonadError QErr m)
|
||||
=> (Field -> m s)
|
||||
-> QualifiedFunction -> FuncArgSeq
|
||||
-> Field
|
||||
-> m (RS.AnnFnSelG s UnresolvedVal)
|
||||
fromFuncQueryField fn qf argSeq fld = fieldAsPath fld $ do
|
||||
funcArgsM <- withArgM (_fArguments fld) "args" $ parseFunctionArgs argSeq
|
||||
let funcArgs = maybe [] (map UVPG) funcArgsM
|
||||
RS.AnnFnSel qf funcArgs <$> fn fld
|
||||
|
||||
convertFuncQuerySimple
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has FieldMap r
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
)
|
||||
=> FuncQOpCtx -> Field -> m QueryRootFldUnresolved
|
||||
convertFuncQuerySimple funcOpCtx fld =
|
||||
withPathK "selectionSet" $ fmap QRFFnSimple $
|
||||
fromFuncQueryField (fromField qt permFilter permLimit) qf argSeq fld
|
||||
where
|
||||
FuncQOpCtx qt _ permFilter permLimit qf argSeq = funcOpCtx
|
||||
|
||||
convertFuncQueryAgg
|
||||
:: ( MonadError QErr m
|
||||
, MonadReader r m
|
||||
, Has FieldMap r
|
||||
, Has OrdByCtx r
|
||||
, Has SQLGenCtx r
|
||||
)
|
||||
=> FuncQOpCtx -> Field -> m QueryRootFldUnresolved
|
||||
convertFuncQueryAgg funcOpCtx fld =
|
||||
withPathK "selectionSet" $ fmap QRFFnAgg $
|
||||
fromFuncQueryField (fromAggField qt permFilter permLimit) qf argSeq fld
|
||||
where
|
||||
FuncQOpCtx qt _ permFilter permLimit qf argSeq = funcOpCtx
|
||||
|
||||
data QueryRootFldAST v
|
||||
= QRFPk !(RS.AnnSimpleSelG v)
|
||||
| QRFSimple !(RS.AnnSimpleSelG v)
|
||||
| QRFAgg !(RS.AnnAggSelG v)
|
||||
| QRFFnSimple !(RS.AnnFnSelSimpleG v)
|
||||
| QRFFnAgg !(RS.AnnFnSelAggG v)
|
||||
deriving (Show, Eq)
|
||||
|
||||
type QueryRootFldUnresolved = QueryRootFldAST UnresolvedVal
|
||||
type QueryRootFldResolved = QueryRootFldAST S.SQLExp
|
||||
|
||||
traverseQueryRootFldAST
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> QueryRootFldAST a
|
||||
-> f (QueryRootFldAST b)
|
||||
traverseQueryRootFldAST f = \case
|
||||
QRFPk s -> QRFPk <$> RS.traverseAnnSimpleSel f s
|
||||
QRFSimple s -> QRFSimple <$> RS.traverseAnnSimpleSel f s
|
||||
QRFAgg s -> QRFAgg <$> RS.traverseAnnAggSel f s
|
||||
QRFFnSimple s -> QRFFnSimple <$> RS.traverseAnnFnSimple f s
|
||||
QRFFnAgg s -> QRFFnAgg <$> RS.traverseAnnFnAgg f s
|
||||
|
||||
toPGQuery :: QueryRootFldResolved -> Q.Query
|
||||
toPGQuery = \case
|
||||
QRFPk s -> RS.selectQuerySQL True s
|
||||
QRFSimple s -> RS.selectQuerySQL False s
|
||||
QRFAgg s -> RS.selectAggQuerySQL s
|
||||
QRFFnSimple s -> RS.mkFuncSelectSimple s
|
||||
QRFFnAgg s -> RS.mkFuncSelectAgg s
|
||||
|
@ -64,7 +64,7 @@ instance Has TypeMap RemoteGCtx where
|
||||
getter = _rgTypes
|
||||
modifier f ctx = ctx { _rgTypes = f $ _rgTypes ctx }
|
||||
|
||||
type SelField = Either PGColInfo (RelInfo, Bool, AnnBoolExpSQL, Maybe Int, Bool)
|
||||
type SelField = Either PGColInfo (RelInfo, Bool, AnnBoolExpPartialSQL, Maybe Int, Bool)
|
||||
|
||||
qualObjectToName :: (ToTxt a) => QualifiedObject a -> G.Name
|
||||
qualObjectToName = G.Name . snakeCaseQualObject
|
||||
@ -1402,9 +1402,9 @@ getRootFldsRole'
|
||||
-> FieldInfoMap
|
||||
-> [FunctionInfo]
|
||||
-> Maybe ([T.Text], Bool) -- insert perm
|
||||
-> Maybe (AnnBoolExpSQL, Maybe Int, [T.Text], Bool) -- select filter
|
||||
-> Maybe ([PGCol], PreSetCols, AnnBoolExpSQL, [T.Text]) -- update filter
|
||||
-> Maybe (AnnBoolExpSQL, [T.Text]) -- delete filter
|
||||
-> Maybe (AnnBoolExpPartialSQL, Maybe Int, [T.Text], Bool) -- select filter
|
||||
-> Maybe ([PGCol], PreSetColsPartial, AnnBoolExpPartialSQL, [T.Text]) -- update filter
|
||||
-> Maybe (AnnBoolExpPartialSQL, [T.Text]) -- delete filter
|
||||
-> Maybe ViewInfo
|
||||
-> RootFlds
|
||||
getRootFldsRole' tn primCols constraints fields funcs insM selM updM delM viM =
|
||||
@ -1661,7 +1661,7 @@ mkGCtxMapTable tableCache funcCache tabInfo = do
|
||||
(Just (validColNames, mempty, noFilter, [])) (Just (noFilter, []))
|
||||
viewInfo
|
||||
|
||||
noFilter :: AnnBoolExpSQL
|
||||
noFilter :: AnnBoolExpPartialSQL
|
||||
noFilter = annBoolExpTrue
|
||||
|
||||
checkSchemaConflicts
|
||||
|
@ -3,58 +3,53 @@ module Hasura.GraphQL.Transport.HTTP
|
||||
) where
|
||||
|
||||
import qualified Data.ByteString.Lazy as BL
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Network.HTTP.Client as HTTP
|
||||
import qualified Network.HTTP.Types as N
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Schema
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types
|
||||
|
||||
import qualified Hasura.GraphQL.Execute as E
|
||||
import qualified Hasura.GraphQL.Resolve as R
|
||||
import qualified Hasura.GraphQL.Validate as V
|
||||
|
||||
runGQ
|
||||
:: (MonadIO m, MonadError QErr m)
|
||||
=> Q.PGPool -> Q.TxIsolation
|
||||
=> PGExecCtx
|
||||
-> UserInfo
|
||||
-> SQLGenCtx
|
||||
-> E.PlanCache
|
||||
-> SchemaCache
|
||||
-> SchemaCacheVer
|
||||
-> HTTP.Manager
|
||||
-> [N.Header]
|
||||
-> GraphQLRequest
|
||||
-> GQLReqUnparsed
|
||||
-> BL.ByteString -- this can be removed when we have a pretty-printer
|
||||
-> m EncJSON
|
||||
runGQ pool isoL userInfo sqlGenCtx sc manager reqHdrs req rawReq = do
|
||||
execPlan <- E.getExecPlan userInfo sc req
|
||||
runGQ pgExecCtx userInfo sqlGenCtx planCache sc scVer
|
||||
manager reqHdrs req rawReq = do
|
||||
execPlan <- E.getResolvedExecPlan pgExecCtx planCache
|
||||
userInfo sqlGenCtx sc scVer req
|
||||
case execPlan of
|
||||
E.GExPHasura gCtx rootSelSet ->
|
||||
runHasuraGQ pool isoL userInfo sqlGenCtx gCtx rootSelSet
|
||||
E.GExPHasura resolvedOp ->
|
||||
runHasuraGQ pgExecCtx userInfo resolvedOp
|
||||
E.GExPRemote rsi opDef ->
|
||||
E.execRemoteGQ manager userInfo reqHdrs rawReq rsi opDef
|
||||
|
||||
runHasuraGQ
|
||||
:: (MonadIO m, MonadError QErr m)
|
||||
=> Q.PGPool
|
||||
-> Q.TxIsolation
|
||||
=> PGExecCtx
|
||||
-> UserInfo
|
||||
-> SQLGenCtx
|
||||
-> GCtx
|
||||
-> V.RootSelSet
|
||||
-> E.ExecOp
|
||||
-> m EncJSON
|
||||
runHasuraGQ pool isoL userInfo sqlGenCtx gCtx rootSelSet = do
|
||||
tx <- case rootSelSet of
|
||||
V.RQuery selSet ->
|
||||
return $ R.resolveQuerySelSet userInfo gCtx sqlGenCtx selSet
|
||||
V.RMutation selSet ->
|
||||
return $ R.resolveMutSelSet userInfo gCtx sqlGenCtx selSet
|
||||
V.RSubscription _ ->
|
||||
runHasuraGQ pgExecCtx userInfo resolvedOp = do
|
||||
respE <- liftIO $ runExceptT $ case resolvedOp of
|
||||
E.ExOpQuery tx ->
|
||||
runLazyTx' pgExecCtx tx
|
||||
E.ExOpMutation tx ->
|
||||
runLazyTx pgExecCtx $ withUserInfo userInfo tx
|
||||
E.ExOpSubs _ ->
|
||||
throw400 UnexpectedPayload
|
||||
"subscriptions are not supported over HTTP, use websockets instead"
|
||||
resp <- liftIO (runExceptT $ runTx tx) >>= liftEither
|
||||
resp <- liftEither respE
|
||||
return $ encodeGQResp $ GQSuccess $ encJToLBS resp
|
||||
where
|
||||
runTx tx = runLazyTx pool isoL $ withUserInfo userInfo tx
|
||||
|
@ -1,6 +1,10 @@
|
||||
module Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
( GraphQLRequest(..)
|
||||
, GraphQLQuery(..)
|
||||
( GQLReq(..)
|
||||
, GQLReqUnparsed
|
||||
, GQLReqParsed
|
||||
, toParsed
|
||||
, GQLQueryText
|
||||
, GQLExecDoc(..)
|
||||
, OperationName(..)
|
||||
, VariableValues
|
||||
, encodeGQErr
|
||||
@ -10,6 +14,7 @@ module Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
) where
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Utils
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types
|
||||
|
||||
@ -21,41 +26,56 @@ import qualified Data.HashMap.Strict as Map
|
||||
import qualified Language.GraphQL.Draft.Parser as G
|
||||
import qualified Language.GraphQL.Draft.Syntax as G
|
||||
|
||||
newtype GraphQLQuery
|
||||
= GraphQLQuery { unGraphQLQuery :: [G.ExecutableDefinition] }
|
||||
deriving (Show, Eq, Hashable)
|
||||
newtype GQLExecDoc
|
||||
= GQLExecDoc { unGQLExecDoc :: [G.ExecutableDefinition] }
|
||||
deriving (Ord, Show, Eq, Hashable)
|
||||
|
||||
instance J.FromJSON GraphQLQuery where
|
||||
parseJSON = J.withText "GraphQLQuery" $ \t ->
|
||||
instance J.FromJSON GQLExecDoc where
|
||||
parseJSON = J.withText "GQLExecDoc" $ \t ->
|
||||
case G.parseExecutableDoc t of
|
||||
Left _ -> fail "parsing the graphql query failed"
|
||||
Right q -> return $ GraphQLQuery $ G.getExecutableDefinitions q
|
||||
Right q -> return $ GQLExecDoc $ G.getExecutableDefinitions q
|
||||
|
||||
instance J.ToJSON GraphQLQuery where
|
||||
instance J.ToJSON GQLExecDoc where
|
||||
-- TODO, add pretty printer in graphql-parser
|
||||
toJSON _ = J.String "toJSON not implemented for GraphQLQuery"
|
||||
toJSON _ = J.String "toJSON not implemented for GQLExecDoc"
|
||||
|
||||
newtype OperationName
|
||||
= OperationName { _unOperationName :: G.Name }
|
||||
deriving (Show, Eq, Hashable, J.ToJSON)
|
||||
deriving (Ord, Show, Eq, Hashable, J.ToJSON)
|
||||
|
||||
instance J.FromJSON OperationName where
|
||||
parseJSON v = OperationName . G.Name <$> J.parseJSON v
|
||||
|
||||
type VariableValues = Map.HashMap G.Variable J.Value
|
||||
|
||||
data GraphQLRequest
|
||||
= GraphQLRequest
|
||||
data GQLReq a
|
||||
= GQLReq
|
||||
{ _grOperationName :: !(Maybe OperationName)
|
||||
, _grQuery :: !GraphQLQuery
|
||||
, _grQuery :: !a
|
||||
, _grVariables :: !(Maybe VariableValues)
|
||||
} deriving (Show, Eq, Generic)
|
||||
|
||||
$(J.deriveJSON (J.aesonDrop 3 J.camelCase){J.omitNothingFields=True}
|
||||
''GraphQLRequest
|
||||
''GQLReq
|
||||
)
|
||||
|
||||
instance Hashable GraphQLRequest
|
||||
instance (Hashable a) => Hashable (GQLReq a)
|
||||
|
||||
newtype GQLQueryText
|
||||
= GQLQueryText
|
||||
{ _unGQLQueryText :: Text
|
||||
} deriving (Show, Eq, J.FromJSON, J.ToJSON, Hashable)
|
||||
|
||||
type GQLReqUnparsed = GQLReq GQLQueryText
|
||||
type GQLReqParsed = GQLReq GQLExecDoc
|
||||
|
||||
toParsed :: (MonadError QErr m ) => GQLReqUnparsed -> m GQLReqParsed
|
||||
toParsed req = case G.parseExecutableDoc gqlText of
|
||||
Left _ -> withPathK "query" $ throwVE "not a valid graphql query"
|
||||
Right a -> return $ req { _grQuery = GQLExecDoc $ G.getExecutableDefinitions a }
|
||||
where
|
||||
gqlText = _unGQLQueryText $ _grQuery req
|
||||
|
||||
encodeGQErr :: Bool -> QErr -> J.Value
|
||||
encodeGQErr includeInternal qErr =
|
||||
|
@ -3,6 +3,7 @@
|
||||
module Hasura.GraphQL.Transport.WebSocket
|
||||
( createWSServerApp
|
||||
, createWSServerEnv
|
||||
, WSServerEnv
|
||||
) where
|
||||
|
||||
import qualified Control.Concurrent.Async as A
|
||||
@ -27,14 +28,11 @@ import Data.ByteString (ByteString)
|
||||
import qualified Data.IORef as IORef
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.GraphQL.Context (GCtx)
|
||||
import qualified Hasura.GraphQL.Execute as E
|
||||
import qualified Hasura.GraphQL.Resolve as R
|
||||
import qualified Hasura.GraphQL.Resolve.LiveQuery as LQ
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery as LQ
|
||||
import Hasura.GraphQL.Transport.HTTP.Protocol
|
||||
import Hasura.GraphQL.Transport.WebSocket.Protocol
|
||||
import qualified Hasura.GraphQL.Transport.WebSocket.Server as WS
|
||||
import qualified Hasura.GraphQL.Validate as V
|
||||
import qualified Hasura.Logging as L
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types
|
||||
@ -43,11 +41,8 @@ import Hasura.Server.Auth (AuthMode,
|
||||
import Hasura.Server.Cors
|
||||
import Hasura.Server.Utils (bsToTxt)
|
||||
|
||||
-- uniquely identifies an operation
|
||||
type GOperationId = (WS.WSId, OperationId)
|
||||
|
||||
type OperationMap
|
||||
= STMMap.Map OperationId LQ.LiveQuery
|
||||
= STMMap.Map OperationId (LQ.LiveQueryId, Maybe OperationName)
|
||||
|
||||
newtype WsHeaders
|
||||
= WsHeaders { unWsHeaders :: [H.Header] }
|
||||
@ -70,7 +65,6 @@ data WSConnData
|
||||
, _wscOpMap :: !OperationMap
|
||||
}
|
||||
|
||||
type LiveQueryMap = LQ.LiveQueryMap GOperationId
|
||||
type WSServer = WS.WSServer WSConnData
|
||||
|
||||
type WSConn = WS.WSConn WSConnData
|
||||
@ -120,13 +114,14 @@ instance L.ToEngineLog WSLog where
|
||||
data WSServerEnv
|
||||
= WSServerEnv
|
||||
{ _wseLogger :: !L.Logger
|
||||
, _wseServer :: !WSServer
|
||||
, _wseRunTx :: !LQ.TxRunner
|
||||
, _wseLiveQMap :: !LiveQueryMap
|
||||
, _wseGCtxMap :: !(IORef.IORef SchemaCache)
|
||||
, _wseRunTx :: !PGExecCtx
|
||||
, _wseLiveQMap :: !LQ.LiveQueriesState
|
||||
, _wseGCtxMap :: !(IORef.IORef (SchemaCache, SchemaCacheVer))
|
||||
, _wseHManager :: !H.Manager
|
||||
, _wseCorsPolicy :: !CorsPolicy
|
||||
, _wseSQLCtx :: !SQLGenCtx
|
||||
, _wseQueryCache :: !E.PlanCache
|
||||
, _wseServer :: !WSServer
|
||||
}
|
||||
|
||||
onConn :: L.Logger -> CorsPolicy -> WS.OnConnH WSConnData
|
||||
@ -217,36 +212,32 @@ onStart serverEnv wsConn (StartMsg opId q) msgRaw = catchAndIgnore $ do
|
||||
let connErr = "start received before the connection is initialised"
|
||||
withComplete $ sendConnErr connErr
|
||||
|
||||
sc <- liftIO $ IORef.readIORef gCtxMapRef
|
||||
execPlanE <- runExceptT $ E.getExecPlan userInfo sc q
|
||||
(sc, scVer) <- liftIO $ IORef.readIORef gCtxMapRef
|
||||
execPlanE <- runExceptT $ E.getResolvedExecPlan pgExecCtx
|
||||
planCache userInfo sqlGenCtx sc scVer q
|
||||
execPlan <- either (withComplete . preExecErr) return execPlanE
|
||||
case execPlan of
|
||||
E.GExPHasura gCtx rootSelSet ->
|
||||
runHasuraGQ userInfo gCtx rootSelSet
|
||||
E.GExPHasura resolvedOp ->
|
||||
runHasuraGQ userInfo resolvedOp
|
||||
E.GExPRemote rsi opDef ->
|
||||
runRemoteGQ userInfo reqHdrs opDef rsi
|
||||
where
|
||||
runHasuraGQ :: UserInfo -> GCtx -> V.RootSelSet -> ExceptT () IO ()
|
||||
runHasuraGQ userInfo gCtx rootSelSet =
|
||||
case rootSelSet of
|
||||
V.RQuery selSet ->
|
||||
execQueryOrMut $ withUserInfo userInfo $
|
||||
R.resolveQuerySelSet userInfo gCtx sqlGenCtx selSet
|
||||
V.RMutation selSet ->
|
||||
execQueryOrMut $ withUserInfo userInfo $
|
||||
R.resolveMutSelSet userInfo gCtx sqlGenCtx selSet
|
||||
V.RSubscription fld -> do
|
||||
let tx = withUserInfo userInfo $
|
||||
R.resolveSubsFld userInfo gCtx sqlGenCtx fld
|
||||
let lq = LQ.LiveQuery userInfo q
|
||||
liftIO $ STM.atomically $ STMMap.insert lq opId opMap
|
||||
liftIO $ LQ.addLiveQuery runTx lqMap lq
|
||||
tx (wsId, opId) liveQOnChange
|
||||
logOpEv ODStarted
|
||||
runHasuraGQ :: UserInfo -> E.ExecOp -> ExceptT () IO ()
|
||||
runHasuraGQ userInfo = \case
|
||||
E.ExOpQuery opTx ->
|
||||
execQueryOrMut $ runLazyTx' pgExecCtx opTx
|
||||
E.ExOpMutation opTx ->
|
||||
execQueryOrMut $ runLazyTx pgExecCtx $
|
||||
withUserInfo userInfo opTx
|
||||
E.ExOpSubs lqOp -> do
|
||||
lqId <- liftIO $ LQ.addLiveQuery lqMap lqOp liveQOnChange
|
||||
liftIO $ STM.atomically $
|
||||
STMMap.insert (lqId, _grOperationName q) opId opMap
|
||||
logOpEv ODStarted
|
||||
|
||||
execQueryOrMut tx = do
|
||||
execQueryOrMut action = do
|
||||
logOpEv ODStarted
|
||||
resp <- liftIO $ runTx tx
|
||||
resp <- liftIO $ runExceptT action
|
||||
either postExecErr sendSuccResp resp
|
||||
sendCompleted
|
||||
|
||||
@ -271,9 +262,9 @@ onStart serverEnv wsConn (StartMsg opId q) msgRaw = catchAndIgnore $ do
|
||||
either postExecErr sendSuccResp resp
|
||||
sendCompleted
|
||||
|
||||
WSServerEnv logger _ runTx lqMap gCtxMapRef httpMgr _ sqlGenCtx = serverEnv
|
||||
WSServerEnv logger pgExecCtx lqMap gCtxMapRef httpMgr _
|
||||
sqlGenCtx planCache _ = serverEnv
|
||||
|
||||
wsId = WS.getWSId wsConn
|
||||
WSConnData userInfoR opMap = WS.getData wsConn
|
||||
|
||||
logOpEv opDet =
|
||||
@ -339,16 +330,14 @@ onStop serverEnv wsConn (StopMsg opId) = do
|
||||
-- probably wrap the whole thing in a single tx?
|
||||
opM <- liftIO $ STM.atomically $ STMMap.lookup opId opMap
|
||||
case opM of
|
||||
Just liveQ -> do
|
||||
let opNameM = _grOperationName $ LQ._lqRequest liveQ
|
||||
Just (lqId, opNameM) -> do
|
||||
logWSEvent logger wsConn $ EOperation opId opNameM ODStopped
|
||||
LQ.removeLiveQuery lqMap liveQ (wsId, opId)
|
||||
LQ.removeLiveQuery lqMap lqId
|
||||
Nothing -> return ()
|
||||
STM.atomically $ STMMap.delete opId opMap
|
||||
where
|
||||
logger = _wseLogger serverEnv
|
||||
lqMap = _wseLiveQMap serverEnv
|
||||
wsId = WS.getWSId wsConn
|
||||
opMap = _wscOpMap $ WS.getData wsConn
|
||||
|
||||
logWSEvent
|
||||
@ -399,28 +388,34 @@ onConnInit logger manager wsConn authMode connParamsM = do
|
||||
|
||||
onClose
|
||||
:: L.Logger
|
||||
-> LiveQueryMap
|
||||
-> LQ.LiveQueriesState
|
||||
-> WS.ConnectionException
|
||||
-> WSConn
|
||||
-> IO ()
|
||||
onClose logger lqMap _ wsConn = do
|
||||
logWSEvent logger wsConn EClosed
|
||||
operations <- STM.atomically $ ListT.toList $ STMMap.listT opMap
|
||||
void $ A.forConcurrently operations $ \(opId, liveQ) ->
|
||||
LQ.removeLiveQuery lqMap liveQ (wsId, opId)
|
||||
void $ A.forConcurrently operations $ \(_, (lqId, _)) ->
|
||||
LQ.removeLiveQuery lqMap lqId
|
||||
where
|
||||
wsId = WS.getWSId wsConn
|
||||
opMap = _wscOpMap $ WS.getData wsConn
|
||||
|
||||
createWSServerEnv
|
||||
:: L.Logger
|
||||
-> H.Manager -> SQLGenCtx -> IORef.IORef SchemaCache
|
||||
-> LQ.TxRunner -> CorsPolicy -> IO WSServerEnv
|
||||
createWSServerEnv logger httpManager sqlGenCtx cacheRef runTx corsPolicy = do
|
||||
(wsServer, lqMap) <-
|
||||
STM.atomically $ (,) <$> WS.createWSServer logger <*> LQ.newLiveQueryMap
|
||||
return $ WSServerEnv logger wsServer runTx lqMap cacheRef
|
||||
httpManager corsPolicy sqlGenCtx
|
||||
-> PGExecCtx
|
||||
-> LQ.LiveQueriesState
|
||||
-> IORef.IORef (SchemaCache, SchemaCacheVer)
|
||||
-> H.Manager
|
||||
-> CorsPolicy
|
||||
-> SQLGenCtx
|
||||
-> E.PlanCache
|
||||
-> IO WSServerEnv
|
||||
createWSServerEnv logger pgExecCtx lqState cacheRef httpManager
|
||||
corsPolicy sqlGenCtx planCache = do
|
||||
wsServer <- STM.atomically $ WS.createWSServer logger
|
||||
return $ WSServerEnv logger
|
||||
pgExecCtx lqState cacheRef
|
||||
httpManager corsPolicy sqlGenCtx planCache wsServer
|
||||
|
||||
createWSServerApp :: AuthMode -> WSServerEnv -> WS.ServerApp
|
||||
createWSServerApp authMode serverEnv =
|
||||
|
@ -29,7 +29,7 @@ newtype OperationId
|
||||
data StartMsg
|
||||
= StartMsg
|
||||
{ _smId :: !OperationId
|
||||
, _smPayload :: !GraphQLRequest
|
||||
, _smPayload :: !GQLReqUnparsed
|
||||
} deriving (Show, Eq)
|
||||
$(J.deriveJSON (J.aesonDrop 3 J.snakeCase) ''StartMsg)
|
||||
|
||||
|
@ -1,10 +1,17 @@
|
||||
module Hasura.GraphQL.Validate
|
||||
( validateGQ
|
||||
, showVars
|
||||
, RootSelSet(..)
|
||||
, getTypedOp
|
||||
, GraphQLRequest
|
||||
, QueryParts (..)
|
||||
, getQueryParts
|
||||
, getAnnVarVals
|
||||
|
||||
, VarPGTypes
|
||||
, AnnPGVarVals
|
||||
, getAnnPGVarVals
|
||||
, Field(..)
|
||||
, SelSet
|
||||
) where
|
||||
|
||||
import Data.Has
|
||||
@ -22,6 +29,9 @@ import Hasura.GraphQL.Validate.InputValue
|
||||
import Hasura.GraphQL.Validate.Types
|
||||
import Hasura.RQL.Types
|
||||
|
||||
import Hasura.SQL.Types (PGColType)
|
||||
import Hasura.SQL.Value (PGColValue, parsePGValue)
|
||||
|
||||
data QueryParts
|
||||
= QueryParts
|
||||
{ qpOpDef :: !G.TypedOperationDefinition
|
||||
@ -89,16 +99,47 @@ getAnnVarVals varDefsL inpVals = do
|
||||
annInpValM <- withPathK "variableValues" $
|
||||
mapM (validateInputValue jsonParser ty) inpValM
|
||||
let varValM = annInpValM <|> annDefM
|
||||
onNothing varValM $ throwVE $ "expecting a value for non-null type: "
|
||||
<> G.showGT ty <> " in variableValues"
|
||||
onNothing varValM $ throwVE $
|
||||
"expecting a value for non-nullable variable: " <>
|
||||
showVars [var] <>
|
||||
" of type: " <> G.showGT ty <>
|
||||
" in variableValues"
|
||||
where
|
||||
objTyErrMsg namedTy =
|
||||
"variables can only be defined on input types"
|
||||
<> "(enums, scalars, input objects), but "
|
||||
<> showNamedTy namedTy <> " is an object type"
|
||||
|
||||
showVars :: (Functor f, Foldable f) => f G.Variable -> Text
|
||||
showVars = showNames . fmap G.unVariable
|
||||
showVars :: (Functor f, Foldable f) => f G.Variable -> Text
|
||||
showVars = showNames . fmap G.unVariable
|
||||
|
||||
type VarPGTypes = Map.HashMap G.Variable PGColType
|
||||
type AnnPGVarVals = Map.HashMap G.Variable (PGColType, PGColValue)
|
||||
|
||||
-- this is in similar spirit to getAnnVarVals, however
|
||||
-- here it is much simpler and can get rid of typemap requirement
|
||||
-- combine the two if possible
|
||||
getAnnPGVarVals
|
||||
:: (MonadError QErr m)
|
||||
=> VarPGTypes
|
||||
-> Maybe VariableValues
|
||||
-> m AnnPGVarVals
|
||||
getAnnPGVarVals varTypes varValsM =
|
||||
flip Map.traverseWithKey varTypes $ \varName varType -> do
|
||||
let unexpectedVars = filter
|
||||
(not . (`Map.member` varTypes)) $ Map.keys varVals
|
||||
unless (null unexpectedVars) $
|
||||
throwVE $ "unexpected variables in variableValues: " <>
|
||||
showVars unexpectedVars
|
||||
varVal <- onNothing (Map.lookup varName varVals) $
|
||||
throwVE $ "expecting a value for non-nullable variable: " <>
|
||||
showVars [varName] <>
|
||||
-- TODO: we don't have the graphql type
|
||||
-- " of type: " <> T.pack (show varType) <>
|
||||
" in variableValues"
|
||||
(varType,) <$> runAesonParser (parsePGValue varType) varVal
|
||||
where
|
||||
varVals = fromMaybe Map.empty varValsM
|
||||
|
||||
validateFrag
|
||||
:: (MonadError QErr m, MonadReader r m, Has TypeMap r)
|
||||
@ -156,9 +197,9 @@ validateGQ (QueryParts opDef opRoot fragDefsL varValsM) = do
|
||||
|
||||
getQueryParts
|
||||
:: ( MonadError QErr m, MonadReader GCtx m)
|
||||
=> GraphQLRequest
|
||||
=> GQLReqParsed
|
||||
-> m QueryParts
|
||||
getQueryParts (GraphQLRequest opNameM q varValsM) = do
|
||||
getQueryParts (GQLReq opNameM q varValsM) = do
|
||||
-- get the operation that needs to be evaluated
|
||||
opDef <- getTypedOp opNameM selSets opDefs
|
||||
ctx <- ask
|
||||
@ -172,4 +213,4 @@ getQueryParts (GraphQLRequest opNameM q varValsM) = do
|
||||
onNothing (_gSubRoot ctx) $ throwVE "no subscriptions exist"
|
||||
return $ QueryParts opDef opRoot fragDefsL varValsM
|
||||
where
|
||||
(selSets, opDefs, fragDefsL) = G.partitionExDefs $ unGraphQLQuery q
|
||||
(selSets, opDefs, fragDefsL) = G.partitionExDefs $ unGQLExecDoc q
|
||||
|
@ -106,7 +106,8 @@ dropView vn =
|
||||
|
||||
procSetObj
|
||||
:: (QErrM m)
|
||||
=> TableInfo -> Maybe ColVals -> m (PreSetCols, [Text], [SchemaDependency])
|
||||
=> TableInfo -> Maybe ColVals
|
||||
-> m (PreSetColsPartial, [Text], [SchemaDependency])
|
||||
procSetObj ti mObj = do
|
||||
setColsSQL <- withPathK "set" $
|
||||
fmap HM.fromList $ forM (HM.toList setObj) $ \(pgCol, val) -> do
|
||||
@ -128,7 +129,8 @@ buildInsPermInfo
|
||||
=> TableInfo
|
||||
-> PermDef InsPerm
|
||||
-> m (WithDeps InsPermInfo)
|
||||
buildInsPermInfo tabInfo (PermDef rn (InsPerm chk set mCols) _) = withPathK "permission" $ do
|
||||
buildInsPermInfo tabInfo (PermDef rn (InsPerm chk set mCols) _) =
|
||||
withPathK "permission" $ do
|
||||
(be, beDeps) <- withPathK "check" $
|
||||
-- procBoolExp tn fieldInfoMap (S.QualVar "NEW") chk
|
||||
procBoolExp tn fieldInfoMap chk
|
||||
@ -147,7 +149,9 @@ buildInsPermInfo tabInfo (PermDef rn (InsPerm chk set mCols) _) = withPathK "per
|
||||
|
||||
buildInsInfra :: QualifiedTable -> InsPermInfo -> Q.TxE QErr ()
|
||||
buildInsInfra tn (InsPermInfo _ vn be _ _) = do
|
||||
trigFnQ <- buildInsTrigFn vn tn $ toSQLBoolExp (S.QualVar "NEW") be
|
||||
resolvedBoolExp <- convAnnBoolExpPartialSQL sessVarFromCurrentSetting be
|
||||
trigFnQ <- buildInsTrigFn vn tn $
|
||||
toSQLBoolExp (S.QualVar "NEW") resolvedBoolExp
|
||||
Q.catchE defaultTxErrorHandler $ do
|
||||
-- Create the view
|
||||
Q.unitQ (buildView tn vn) () False
|
||||
|
@ -22,10 +22,8 @@ import Hasura.RQL.GBoolExp
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.Server.Utils
|
||||
import Hasura.SQL.Types
|
||||
import Hasura.SQL.Value (withGeoVal)
|
||||
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
data PermColSpec
|
||||
= PCStar
|
||||
@ -176,7 +174,7 @@ data CreatePermP1Res a
|
||||
procBoolExp
|
||||
:: (QErrM m, CacheRM m)
|
||||
=> QualifiedTable -> FieldInfoMap -> BoolExp
|
||||
-> m (AnnBoolExpSQL, [SchemaDependency])
|
||||
-> m (AnnBoolExpPartialSQL, [SchemaDependency])
|
||||
procBoolExp tn fieldInfoMap be = do
|
||||
abe <- annBoolExp valueParser fieldInfoMap be
|
||||
let deps = getBoolExpDeps tn abe
|
||||
@ -203,20 +201,17 @@ getDependentHeaders :: BoolExp -> [T.Text]
|
||||
getDependentHeaders (BoolExp boolExp) =
|
||||
flip foldMap boolExp $ \(ColExp _ v) -> getDepHeadersFromVal v
|
||||
|
||||
valueParser :: (MonadError QErr m) => PGColType -> Value -> m S.SQLExp
|
||||
valueParser
|
||||
:: (MonadError QErr m)
|
||||
=> PGColType -> Value -> m PartialSQLExp
|
||||
valueParser columnType = \case
|
||||
-- When it is a special variable
|
||||
val@(String t)
|
||||
| isUserVar t -> return $ fromCurSess t
|
||||
| isReqUserId t -> return $ fromCurSess userIdHeader
|
||||
| otherwise -> txtRHSBuilder columnType val
|
||||
| isUserVar t -> return $ PSESessVar columnType t
|
||||
| isReqUserId t -> return $ PSESessVar columnType userIdHeader
|
||||
| otherwise -> PSESQLExp <$> txtRHSBuilder columnType val
|
||||
-- Typical value as Aeson's value
|
||||
val -> txtRHSBuilder columnType val
|
||||
where
|
||||
curSess = S.SEUnsafe "current_setting('hasura.user')::json"
|
||||
fromCurSess hdr = withAnnTy $ withGeoVal columnType $
|
||||
S.SEOpApp (S.SQLOp "->>") [curSess, S.SELit $ T.toLower hdr]
|
||||
withAnnTy v = S.SETyAnn v $ S.AnnType $ T.pack $ show columnType
|
||||
val -> PSESQLExp <$> txtRHSBuilder columnType val
|
||||
|
||||
injectDefaults :: QualifiedTable -> QualifiedTable -> Q.Query
|
||||
injectDefaults qv qt =
|
||||
@ -355,13 +350,3 @@ runDropPerm defn = do
|
||||
permInfo <- buildDropPermP1Res defn
|
||||
dropPermP2 defn permInfo
|
||||
return successMsg
|
||||
|
||||
-- instance (IsPerm a) => HDBQuery (DropPerm a) where
|
||||
|
||||
-- type Phase1Res (DropPerm a) = DropPermP1Res a
|
||||
|
||||
-- phaseOne = buildDropPermP1Res
|
||||
|
||||
-- phaseTwo dp p1Res = dropPermP2 dp p1Res >> return successMsg
|
||||
|
||||
-- schemaCachePolicy = SCPReload
|
||||
|
@ -16,6 +16,7 @@ module Hasura.RQL.DDL.QueryTemplate
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.DML.Internal (sessVarFromCurrentSetting)
|
||||
import Hasura.RQL.GBoolExp (txtRHSBuilder)
|
||||
import Hasura.RQL.Types
|
||||
import Hasura.SQL.Types
|
||||
@ -88,9 +89,9 @@ mkSelQ (DMLQuery tn (SelectG c w o lim offset)) = do
|
||||
|
||||
data QueryTP1
|
||||
= QTP1Insert R.InsertQueryP1
|
||||
| QTP1Select R.AnnSel
|
||||
| QTP1Update R.UpdateQueryP1
|
||||
| QTP1Delete R.DeleteQueryP1
|
||||
| QTP1Select R.AnnSimpleSel
|
||||
| QTP1Update R.AnnUpd
|
||||
| QTP1Delete R.AnnDel
|
||||
| QTP1Count R.CountQueryP1
|
||||
| QTP1Bulk [QueryTP1]
|
||||
deriving (Show, Eq)
|
||||
@ -100,11 +101,16 @@ validateTQuery
|
||||
=> QueryT
|
||||
-> m QueryTP1
|
||||
validateTQuery qt = withPathK "args" $ case qt of
|
||||
QTInsert q -> QTP1Insert <$> R.convInsertQuery decodeInsObjs validateParam q
|
||||
QTSelect q -> QTP1Select <$> (mkSelQ q >>= R.convSelectQuery validateParam)
|
||||
QTUpdate q -> QTP1Update <$> R.validateUpdateQueryWith validateParam q
|
||||
QTDelete q -> QTP1Delete <$> R.validateDeleteQWith validateParam q
|
||||
QTCount q -> QTP1Count <$> R.validateCountQWith validateParam q
|
||||
QTInsert q -> QTP1Insert <$>
|
||||
R.convInsertQuery decodeInsObjs sessVarFromCurrentSetting validateParam q
|
||||
QTSelect q -> QTP1Select <$>
|
||||
(mkSelQ q >>= R.convSelectQuery sessVarFromCurrentSetting validateParam)
|
||||
QTUpdate q -> QTP1Update <$>
|
||||
R.validateUpdateQueryWith sessVarFromCurrentSetting validateParam q
|
||||
QTDelete q -> QTP1Delete <$>
|
||||
R.validateDeleteQWith sessVarFromCurrentSetting validateParam q
|
||||
QTCount q -> QTP1Count <$>
|
||||
R.validateCountQWith sessVarFromCurrentSetting validateParam q
|
||||
QTBulk q -> QTP1Bulk <$> mapM validateTQuery q
|
||||
where
|
||||
decodeInsObjs val = do
|
||||
|
@ -73,10 +73,11 @@ mkSQLCount (CountQueryP1 tn (permFltr, mWc) mDistCols) =
|
||||
-- SELECT count(*) FROM (SELECT * FROM .. WHERE ..) r;
|
||||
validateCountQWith
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> (PGColType -> Value -> m S.SQLExp)
|
||||
=> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> CountQuery
|
||||
-> m CountQueryP1
|
||||
validateCountQWith prepValBuilder (CountQuery qt mDistCols mWhere) = do
|
||||
validateCountQWith sessVarBldr prepValBldr (CountQuery qt mDistCols mWhere) = do
|
||||
tableInfo <- askTabInfo qt
|
||||
|
||||
-- Check if select is allowed
|
||||
@ -93,11 +94,14 @@ validateCountQWith prepValBuilder (CountQuery qt mDistCols mWhere) = do
|
||||
-- convert the where clause
|
||||
annSQLBoolExp <- forM mWhere $ \be ->
|
||||
withPathK "where" $
|
||||
convBoolExp' colInfoMap selPerm be prepValBuilder
|
||||
convBoolExp colInfoMap selPerm be sessVarBldr prepValBldr
|
||||
|
||||
resolvedSelFltr <- convAnnBoolExpPartialSQL sessVarBldr $
|
||||
spiFilter selPerm
|
||||
|
||||
return $ CountQueryP1
|
||||
qt
|
||||
(spiFilter selPerm, annSQLBoolExp)
|
||||
(resolvedSelFltr, annSQLBoolExp)
|
||||
mDistCols
|
||||
where
|
||||
selNecessaryMsg =
|
||||
@ -110,7 +114,7 @@ validateCountQ
|
||||
:: (QErrM m, UserInfoM m, CacheRM m, HasSQLGenCtx m)
|
||||
=> CountQuery -> m (CountQueryP1, DS.Seq Q.PrepArg)
|
||||
validateCountQ =
|
||||
liftDMLP1 . validateCountQWith binRHSBuilder
|
||||
liftDMLP1 . validateCountQWith sessVarFromCurrentSetting binRHSBuilder
|
||||
|
||||
countQToTx
|
||||
:: (QErrM m, MonadTx m)
|
||||
|
@ -1,7 +1,9 @@
|
||||
module Hasura.RQL.DML.Delete
|
||||
( validateDeleteQWith
|
||||
, validateDeleteQ
|
||||
, DeleteQueryP1(..)
|
||||
, AnnDelG(..)
|
||||
, traverseAnnDel
|
||||
, AnnDel
|
||||
, deleteQueryToTx
|
||||
, getDeleteDeps
|
||||
, runDelete
|
||||
@ -24,17 +26,32 @@ import Hasura.SQL.Types
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
data DeleteQueryP1
|
||||
= DeleteQueryP1
|
||||
data AnnDelG v
|
||||
= AnnDel
|
||||
{ dqp1Table :: !QualifiedTable
|
||||
, dqp1Where :: !(AnnBoolExpSQL, AnnBoolExpSQL)
|
||||
, dqp1MutFlds :: !MutFlds
|
||||
, dqp1Where :: !(AnnBoolExp v, AnnBoolExp v)
|
||||
, dqp1MutFlds :: !(MutFldsG v)
|
||||
, dqp1AllCols :: ![PGColInfo]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
traverseAnnDel
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnDelG a
|
||||
-> f (AnnDelG b)
|
||||
traverseAnnDel f annUpd =
|
||||
AnnDel tn
|
||||
<$> ((,) <$> traverseAnnBoolExp f whr <*> traverseAnnBoolExp f fltr)
|
||||
<*> traverseMutFlds f mutFlds
|
||||
<*> pure allCols
|
||||
where
|
||||
AnnDel tn (whr, fltr) mutFlds allCols = annUpd
|
||||
|
||||
type AnnDel = AnnDelG S.SQLExp
|
||||
|
||||
mkDeleteCTE
|
||||
:: DeleteQueryP1 -> S.CTE
|
||||
mkDeleteCTE (DeleteQueryP1 tn (fltr, wc) _ _) =
|
||||
:: AnnDel -> S.CTE
|
||||
mkDeleteCTE (AnnDel tn (fltr, wc) _ _) =
|
||||
S.CTEDelete delete
|
||||
where
|
||||
delete = S.SQLDelete tn Nothing tableFltr $ Just S.returningStar
|
||||
@ -42,8 +59,8 @@ mkDeleteCTE (DeleteQueryP1 tn (fltr, wc) _ _) =
|
||||
toSQLBoolExp (S.QualTable tn) $ andAnnBoolExps fltr wc
|
||||
|
||||
getDeleteDeps
|
||||
:: DeleteQueryP1 -> [SchemaDependency]
|
||||
getDeleteDeps (DeleteQueryP1 tn (_, wc) mutFlds allCols) =
|
||||
:: AnnDel -> [SchemaDependency]
|
||||
getDeleteDeps (AnnDel tn (_, wc) mutFlds allCols) =
|
||||
mkParentDep tn : allColDeps <> whereDeps <> retDeps
|
||||
where
|
||||
whereDeps = getBoolExpDeps tn wc
|
||||
@ -53,10 +70,12 @@ getDeleteDeps (DeleteQueryP1 tn (_, wc) mutFlds allCols) =
|
||||
|
||||
validateDeleteQWith
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> (PGColType -> Value -> m S.SQLExp)
|
||||
=> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> DeleteQuery
|
||||
-> m DeleteQueryP1
|
||||
validateDeleteQWith prepValBuilder (DeleteQuery tableName rqlBE mRetCols) = do
|
||||
-> m AnnDel
|
||||
validateDeleteQWith sessVarBldr prepValBldr
|
||||
(DeleteQuery tableName rqlBE mRetCols) = do
|
||||
tableInfo <- askTabInfo tableName
|
||||
|
||||
-- If table is view then check if it deletable
|
||||
@ -82,10 +101,13 @@ validateDeleteQWith prepValBuilder (DeleteQuery tableName rqlBE mRetCols) = do
|
||||
|
||||
-- convert the where clause
|
||||
annSQLBoolExp <- withPathK "where" $
|
||||
convBoolExp' fieldInfoMap selPerm rqlBE prepValBuilder
|
||||
convBoolExp fieldInfoMap selPerm rqlBE sessVarBldr prepValBldr
|
||||
|
||||
return $ DeleteQueryP1 tableName
|
||||
(dpiFilter delPerm, annSQLBoolExp)
|
||||
resolvedDelFltr <- convAnnBoolExpPartialSQL sessVarBldr $
|
||||
dpiFilter delPerm
|
||||
|
||||
return $ AnnDel tableName
|
||||
(resolvedDelFltr, annSQLBoolExp)
|
||||
(mkDefaultMutFlds mAnnRetCols) allCols
|
||||
|
||||
where
|
||||
@ -96,11 +118,11 @@ validateDeleteQWith prepValBuilder (DeleteQuery tableName rqlBE mRetCols) = do
|
||||
|
||||
validateDeleteQ
|
||||
:: (QErrM m, UserInfoM m, CacheRM m, HasSQLGenCtx m)
|
||||
=> DeleteQuery -> m (DeleteQueryP1, DS.Seq Q.PrepArg)
|
||||
=> DeleteQuery -> m (AnnDel, DS.Seq Q.PrepArg)
|
||||
validateDeleteQ =
|
||||
liftDMLP1 . validateDeleteQWith binRHSBuilder
|
||||
liftDMLP1 . validateDeleteQWith sessVarFromCurrentSetting binRHSBuilder
|
||||
|
||||
deleteQueryToTx :: Bool -> (DeleteQueryP1, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
deleteQueryToTx :: Bool -> (AnnDel, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
deleteQueryToTx strfyNum (u, p) =
|
||||
runMutation $ Mutation (dqp1Table u) (deleteCTE, p)
|
||||
(dqp1MutFlds u) (dqp1AllCols u) strfyNum
|
||||
|
@ -3,11 +3,10 @@ module Hasura.RQL.DML.Insert where
|
||||
import Data.Aeson.Types
|
||||
import Instances.TH.Lift ()
|
||||
|
||||
import qualified Data.Aeson.Text as AT
|
||||
import qualified Data.Aeson.Extended as J
|
||||
import qualified Data.HashMap.Strict as HM
|
||||
import qualified Data.HashSet as HS
|
||||
import qualified Data.Sequence as DS
|
||||
import qualified Data.Text.Lazy as LT
|
||||
|
||||
import Hasura.EncJSON
|
||||
import Hasura.Prelude
|
||||
@ -107,11 +106,12 @@ validateInpCols inpCols updColsPerm = forM_ inpCols $ \inpCol ->
|
||||
|
||||
buildConflictClause
|
||||
:: (UserInfoM m, QErrM m)
|
||||
=> TableInfo
|
||||
=> SessVarBldr m
|
||||
-> TableInfo
|
||||
-> [PGCol]
|
||||
-> OnConflict
|
||||
-> m ConflictClauseP1
|
||||
buildConflictClause tableInfo inpCols (OnConflict mTCol mTCons act) =
|
||||
buildConflictClause sessVarBldr tableInfo inpCols (OnConflict mTCol mTCons act) =
|
||||
case (mTCol, mTCons, act) of
|
||||
(Nothing, Nothing, CAIgnore) -> return $ CP1DoNothing Nothing
|
||||
(Just col, Nothing, CAIgnore) -> do
|
||||
@ -124,14 +124,18 @@ buildConflictClause tableInfo inpCols (OnConflict mTCol mTCons act) =
|
||||
"Expecting 'constraint' or 'constraint_on' when the 'action' is 'update'"
|
||||
(Just col, Nothing, CAUpdate) -> do
|
||||
validateCols col
|
||||
(updFiltr, preSet) <- getUpdPerm
|
||||
return $ CP1Update (Column $ getPGCols col) inpCols preSet $
|
||||
toSQLBool updFiltr
|
||||
(updFltr, preSet) <- getUpdPerm
|
||||
resolvedUpdFltr <- convAnnBoolExpPartialSQL sessVarBldr updFltr
|
||||
resolvedPreSet <- mapM (convPartialSQLExp sessVarBldr) preSet
|
||||
return $ CP1Update (Column $ getPGCols col) inpCols resolvedPreSet $
|
||||
toSQLBool resolvedUpdFltr
|
||||
(Nothing, Just cons, CAUpdate) -> do
|
||||
validateConstraint cons
|
||||
(updFiltr, preSet) <- getUpdPerm
|
||||
return $ CP1Update (Constraint cons) inpCols preSet $
|
||||
toSQLBool updFiltr
|
||||
(updFltr, preSet) <- getUpdPerm
|
||||
resolvedUpdFltr <- convAnnBoolExpPartialSQL sessVarBldr updFltr
|
||||
resolvedPreSet <- mapM (convPartialSQLExp sessVarBldr) preSet
|
||||
return $ CP1Update (Constraint cons) inpCols resolvedPreSet $
|
||||
toSQLBool resolvedUpdFltr
|
||||
(Just _, Just _, _) -> throw400 UnexpectedPayload
|
||||
"'constraint' and 'constraint_on' cannot be set at a time"
|
||||
where
|
||||
@ -163,10 +167,11 @@ buildConflictClause tableInfo inpCols (OnConflict mTCol mTCons act) =
|
||||
convInsertQuery
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> (Value -> m [InsObj])
|
||||
-> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> InsertQuery
|
||||
-> m InsertQueryP1
|
||||
convInsertQuery objsParser prepFn (InsertQuery tableName val oC mRetCols) = do
|
||||
convInsertQuery objsParser sessVarBldr prepFn (InsertQuery tableName val oC mRetCols) = do
|
||||
|
||||
insObjs <- objsParser val
|
||||
|
||||
@ -202,8 +207,10 @@ convInsertQuery objsParser prepFn (InsertQuery tableName val oC mRetCols) = do
|
||||
insCols = HM.keys defInsVals
|
||||
insView = ipiView insPerm
|
||||
|
||||
resolvedPreSet <- mapM (convPartialSQLExp sessVarBldr) setInsVals
|
||||
|
||||
insTuples <- withPathK "objects" $ indexedForM insObjs $ \obj ->
|
||||
convObj prepFn defInsVals setInsVals fieldInfoMap obj
|
||||
convObj prepFn defInsVals resolvedPreSet fieldInfoMap obj
|
||||
let sqlExps = map snd insTuples
|
||||
inpCols = HS.toList $ HS.fromList $ concatMap fst insTuples
|
||||
|
||||
@ -212,7 +219,7 @@ convInsertQuery objsParser prepFn (InsertQuery tableName val oC mRetCols) = do
|
||||
unless (isTabUpdatable roleName tableInfo) $ throw400 PermissionDenied $
|
||||
"upsert is not allowed for role " <> roleName
|
||||
<<> " since update permissions are not defined"
|
||||
buildConflictClause tableInfo inpCols c
|
||||
buildConflictClause sessVarBldr tableInfo inpCols c
|
||||
|
||||
return $ InsertQueryP1 tableName insView insCols sqlExps
|
||||
conflictClause mutFlds allCols
|
||||
@ -234,7 +241,9 @@ convInsQ
|
||||
-> m (InsertQueryP1, DS.Seq Q.PrepArg)
|
||||
convInsQ =
|
||||
liftDMLP1 .
|
||||
convInsertQuery (withPathK "objects" . decodeInsObjs) binRHSBuilder
|
||||
convInsertQuery (withPathK "objects" . decodeInsObjs)
|
||||
sessVarFromCurrentSetting
|
||||
binRHSBuilder
|
||||
|
||||
insertP2 :: Bool -> (InsertQueryP1, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
insertP2 strfyNum (u, p) =
|
||||
@ -279,12 +288,10 @@ setConflictCtx conflictCtxM = do
|
||||
q = Q.fromBuilder $ setVar <> setVal
|
||||
Q.unitQE defaultTxErrorHandler q () False
|
||||
where
|
||||
encToText = LT.toStrict . AT.encodeToLazyText
|
||||
|
||||
conflictCtxToJSON (CCDoNothing constrM) =
|
||||
encToText $ InsertTxConflictCtx CAIgnore constrM Nothing
|
||||
J.encodeToStrictText $ InsertTxConflictCtx CAIgnore constrM Nothing
|
||||
conflictCtxToJSON (CCUpdate constr updCols preSet filtr) =
|
||||
encToText $ InsertTxConflictCtx CAUpdate (Just constr) $
|
||||
J.encodeToStrictText $ InsertTxConflictCtx CAUpdate (Just constr) $
|
||||
Just $ toSQLTxt (S.buildUpsertSetExp updCols preSet)
|
||||
<> " " <> toSQLTxt (S.WhereFrag filtr)
|
||||
|
||||
|
@ -158,6 +158,8 @@ fetchRelTabInfo refTabName =
|
||||
-- Internal error
|
||||
modifyErrAndSet500 ("foreign " <> ) $ askTabInfo refTabName
|
||||
|
||||
type SessVarBldr m = PGColType -> SessVar -> m S.SQLExp
|
||||
|
||||
fetchRelDet
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> RelName -> QualifiedTable
|
||||
@ -183,36 +185,68 @@ fetchRelDet relName refTabName = do
|
||||
checkOnColExp
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> SelPermInfo
|
||||
-> SessVarBldr m
|
||||
-> AnnBoolExpFldSQL
|
||||
-> m AnnBoolExpFldSQL
|
||||
checkOnColExp spi annFld = case annFld of
|
||||
checkOnColExp spi sessVarBldr annFld = case annFld of
|
||||
AVCol (PGColInfo cn _ _) _ -> do
|
||||
checkSelOnCol spi cn
|
||||
return annFld
|
||||
AVRel relInfo nesAnn -> do
|
||||
relSPI <- snd <$> fetchRelDet (riName relInfo) (riRTable relInfo)
|
||||
modAnn <- checkSelPerm relSPI nesAnn
|
||||
return $ AVRel relInfo $
|
||||
andAnnBoolExps modAnn $ spiFilter relSPI
|
||||
modAnn <- checkSelPerm relSPI sessVarBldr nesAnn
|
||||
resolvedFltr <- convAnnBoolExpPartialSQL sessVarBldr $ spiFilter relSPI
|
||||
return $ AVRel relInfo $ andAnnBoolExps modAnn resolvedFltr
|
||||
|
||||
convAnnBoolExpPartialSQL
|
||||
:: (Applicative f)
|
||||
=> SessVarBldr f
|
||||
-> AnnBoolExpPartialSQL
|
||||
-> f AnnBoolExpSQL
|
||||
convAnnBoolExpPartialSQL f =
|
||||
traverseAnnBoolExp (convPartialSQLExp f)
|
||||
|
||||
convPartialSQLExp
|
||||
:: (Applicative f)
|
||||
=> SessVarBldr f
|
||||
-> PartialSQLExp
|
||||
-> f S.SQLExp
|
||||
convPartialSQLExp f = \case
|
||||
PSESQLExp sqlExp -> pure sqlExp
|
||||
PSESessVar colTy sessVar -> f colTy sessVar
|
||||
|
||||
sessVarFromCurrentSetting
|
||||
:: (Applicative f) => PGColType -> SessVar -> f S.SQLExp
|
||||
sessVarFromCurrentSetting columnType sessVar =
|
||||
pure $ sessVarFromCurrentSetting' columnType sessVar
|
||||
|
||||
sessVarFromCurrentSetting' :: PGColType -> SessVar -> S.SQLExp
|
||||
sessVarFromCurrentSetting' columnType sessVar =
|
||||
S.withTyAnn columnType $ withGeoVal columnType $
|
||||
S.SEOpApp (S.SQLOp "->>") [curSess, S.SELit $ T.toLower sessVar]
|
||||
where
|
||||
curSess = S.SEUnsafe "current_setting('hasura.user')::json"
|
||||
|
||||
checkSelPerm
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> SelPermInfo
|
||||
-> (PGColType -> SessVar -> m S.SQLExp)
|
||||
-> AnnBoolExpSQL
|
||||
-> m AnnBoolExpSQL
|
||||
checkSelPerm spi =
|
||||
traverse (checkOnColExp spi)
|
||||
checkSelPerm spi sessVarBldr =
|
||||
traverse (checkOnColExp spi sessVarBldr)
|
||||
|
||||
convBoolExp'
|
||||
convBoolExp
|
||||
:: ( UserInfoM m, QErrM m, CacheRM m)
|
||||
=> FieldInfoMap
|
||||
-> SelPermInfo
|
||||
-> BoolExp
|
||||
-> (PGColType -> SessVar -> m S.SQLExp)
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> m AnnBoolExpSQL
|
||||
convBoolExp' cim spi be prepValBuilder = do
|
||||
abe <- annBoolExp prepValBuilder cim be
|
||||
checkSelPerm spi abe
|
||||
convBoolExp cim spi be sessVarBldr prepValBldr = do
|
||||
abe <- annBoolExp prepValBldr cim be
|
||||
checkSelPerm spi sessVarBldr abe
|
||||
|
||||
dmlTxErrorHandler :: Q.PGTxErr -> QErr
|
||||
dmlTxErrorHandler p2Res =
|
||||
|
@ -51,9 +51,9 @@ getParamValue params (TemplateParamConf paramName paramVal) =
|
||||
|
||||
data QueryTProc
|
||||
= QTPInsert !(R.InsertQueryP1, DS.Seq Q.PrepArg)
|
||||
| QTPSelect !(R.AnnSel, DS.Seq Q.PrepArg)
|
||||
| QTPUpdate !(R.UpdateQueryP1, DS.Seq Q.PrepArg)
|
||||
| QTPDelete !(R.DeleteQueryP1, DS.Seq Q.PrepArg)
|
||||
| QTPSelect !(R.AnnSimpleSel, DS.Seq Q.PrepArg)
|
||||
| QTPUpdate !(R.AnnUpd, DS.Seq Q.PrepArg)
|
||||
| QTPDelete !(R.AnnDel, DS.Seq Q.PrepArg)
|
||||
| QTPCount !(RC.CountQueryP1, DS.Seq Q.PrepArg)
|
||||
| QTPBulk ![QueryTProc]
|
||||
deriving (Show, Eq)
|
||||
@ -95,13 +95,21 @@ convQT
|
||||
-> QueryT
|
||||
-> m QueryTProc
|
||||
convQT args qt = case qt of
|
||||
QTInsert q -> fmap QTPInsert $ liftDMLP1 $
|
||||
R.convInsertQuery decodeParam binRHSBuilder q
|
||||
QTSelect q -> fmap QTPSelect $ liftDMLP1 $
|
||||
mkSelQWithArgs q args >>= R.convSelectQuery f
|
||||
QTUpdate q -> fmap QTPUpdate $ liftDMLP1 $ R.validateUpdateQueryWith f q
|
||||
QTDelete q -> fmap QTPDelete $ liftDMLP1 $ R.validateDeleteQWith f q
|
||||
QTCount q -> fmap QTPCount $ liftDMLP1 $ RC.validateCountQWith f q
|
||||
QTInsert q ->
|
||||
fmap QTPInsert $ liftDMLP1 $
|
||||
R.convInsertQuery decodeParam sessVarFromCurrentSetting binRHSBuilder q
|
||||
QTSelect q ->
|
||||
fmap QTPSelect $ liftDMLP1 $ mkSelQWithArgs q args
|
||||
>>= R.convSelectQuery sessVarFromCurrentSetting f
|
||||
QTUpdate q ->
|
||||
fmap QTPUpdate $ liftDMLP1 $
|
||||
R.validateUpdateQueryWith sessVarFromCurrentSetting f q
|
||||
QTDelete q ->
|
||||
fmap QTPDelete $ liftDMLP1 $
|
||||
R.validateDeleteQWith sessVarFromCurrentSetting f q
|
||||
QTCount q ->
|
||||
fmap QTPCount $ liftDMLP1 $
|
||||
RC.validateCountQWith sessVarFromCurrentSetting f q
|
||||
QTBulk q -> fmap QTPBulk $ mapM (convQT args) q
|
||||
where
|
||||
decodeParam val = do
|
||||
|
@ -9,13 +9,35 @@ import Hasura.SQL.Types
|
||||
import qualified Data.Text as T
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
data MutFld
|
||||
data MutFldG v
|
||||
= MCount
|
||||
| MExp !T.Text
|
||||
| MRet ![(FieldName, AnnFld)]
|
||||
| MRet ![(FieldName, AnnFldG v)]
|
||||
deriving (Show, Eq)
|
||||
|
||||
type MutFlds = [(T.Text, MutFld)]
|
||||
traverseMutFld
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> MutFldG a
|
||||
-> f (MutFldG b)
|
||||
traverseMutFld f = \case
|
||||
MCount -> pure MCount
|
||||
MExp t -> pure $ MExp t
|
||||
MRet flds -> MRet <$> traverse (traverse (traverseAnnFld f)) flds
|
||||
|
||||
type MutFld = MutFldG S.SQLExp
|
||||
|
||||
type MutFldsG v = [(T.Text, MutFldG v)]
|
||||
|
||||
traverseMutFlds
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> MutFldsG a
|
||||
-> f (MutFldsG b)
|
||||
traverseMutFlds f =
|
||||
traverse (traverse (traverseMutFld f))
|
||||
|
||||
type MutFlds = MutFldsG S.SQLExp
|
||||
|
||||
hasNestedFld :: MutFlds -> Bool
|
||||
hasNestedFld = any isNestedMutFld
|
||||
|
@ -1,9 +1,12 @@
|
||||
module Hasura.RQL.DML.Select
|
||||
( selectP2
|
||||
, selectAggP2
|
||||
, funcQueryTx
|
||||
, selectQuerySQL
|
||||
, selectAggQuerySQL
|
||||
, mkFuncSelectSimple
|
||||
, mkFuncSelectAgg
|
||||
, convSelectQuery
|
||||
, getSelectDeps
|
||||
, asSingleRowJsonResp
|
||||
, module Hasura.RQL.DML.Select.Internal
|
||||
, runSelect
|
||||
)
|
||||
@ -103,10 +106,11 @@ resolveStar fim spi (SelectG selCols mWh mOb mLt mOf) = do
|
||||
|
||||
convOrderByElem
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> (FieldInfoMap, SelPermInfo)
|
||||
=> SessVarBldr m
|
||||
-> (FieldInfoMap, SelPermInfo)
|
||||
-> OrderByCol
|
||||
-> m AnnObCol
|
||||
convOrderByElem (flds, spi) = \case
|
||||
convOrderByElem sessVarBldr (flds, spi) = \case
|
||||
OCPG fldName -> do
|
||||
fldInfo <- askFieldInfo flds fldName
|
||||
case fldInfo of
|
||||
@ -137,17 +141,19 @@ convOrderByElem (flds, spi) = \case
|
||||
," and can't be used in 'order_by'"
|
||||
]
|
||||
(relFim, relSpi) <- fetchRelDet (riName relInfo) (riRTable relInfo)
|
||||
AOCObj relInfo (spiFilter relSpi) <$>
|
||||
convOrderByElem (relFim, relSpi) rest
|
||||
resolvedSelFltr <- convAnnBoolExpPartialSQL sessVarBldr $ spiFilter relSpi
|
||||
AOCObj relInfo resolvedSelFltr <$>
|
||||
convOrderByElem sessVarBldr (relFim, relSpi) rest
|
||||
|
||||
convSelectQ
|
||||
:: (UserInfoM m, QErrM m, CacheRM m, HasSQLGenCtx m)
|
||||
=> FieldInfoMap -- Table information of current table
|
||||
-> SelPermInfo -- Additional select permission info
|
||||
-> SelectQExt -- Given Select Query
|
||||
-> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> m AnnSel
|
||||
convSelectQ fieldInfoMap selPermInfo selQ prepValBuilder = do
|
||||
-> m AnnSimpleSel
|
||||
convSelectQ fieldInfoMap selPermInfo selQ sessVarBldr prepValBldr = do
|
||||
|
||||
annFlds <- withPathK "columns" $
|
||||
indexedForM (sqColumns selQ) $ \case
|
||||
@ -155,7 +161,8 @@ convSelectQ fieldInfoMap selPermInfo selQ prepValBuilder = do
|
||||
colInfo <- convExtSimple fieldInfoMap selPermInfo pgCol
|
||||
return (fromPGCol pgCol, FCol colInfo Nothing)
|
||||
(ECRel relName mAlias relSelQ) -> do
|
||||
annRel <- convExtRel fieldInfoMap relName mAlias relSelQ prepValBuilder
|
||||
annRel <- convExtRel fieldInfoMap relName mAlias
|
||||
relSelQ sessVarBldr prepValBldr
|
||||
return ( fromRel $ fromMaybe relName mAlias
|
||||
, either FObj FArr annRel
|
||||
)
|
||||
@ -165,11 +172,11 @@ convSelectQ fieldInfoMap selPermInfo selQ prepValBuilder = do
|
||||
-- Convert where clause
|
||||
wClause <- forM (sqWhere selQ) $ \be ->
|
||||
withPathK "where" $
|
||||
convBoolExp' fieldInfoMap selPermInfo be prepValBuilder
|
||||
convBoolExp fieldInfoMap selPermInfo be sessVarBldr prepValBldr
|
||||
|
||||
annOrdByML <- forM (sqOrderBy selQ) $ \(OrderByExp obItems) ->
|
||||
withPathK "order_by" $ indexedForM obItems $ mapM $
|
||||
convOrderByElem (fieldInfoMap, selPermInfo)
|
||||
convOrderByElem sessVarBldr (fieldInfoMap, selPermInfo)
|
||||
|
||||
let annOrdByM = NE.nonEmpty =<< annOrdByML
|
||||
|
||||
@ -177,8 +184,11 @@ convSelectQ fieldInfoMap selPermInfo selQ prepValBuilder = do
|
||||
withPathK "limit" $ mapM_ onlyPositiveInt mQueryLimit
|
||||
withPathK "offset" $ mapM_ onlyPositiveInt mQueryOffset
|
||||
|
||||
resolvedSelFltr <- convAnnBoolExpPartialSQL sessVarBldr $
|
||||
spiFilter selPermInfo
|
||||
|
||||
let tabFrom = TableFrom (spiTable selPermInfo) Nothing
|
||||
tabPerm = TablePerm (spiFilter selPermInfo) mPermLimit
|
||||
tabPerm = TablePerm resolvedSelFltr mPermLimit
|
||||
tabArgs = TableArgs wClause annOrdByM mQueryLimit
|
||||
(S.intToSQLExp <$> mQueryOffset) Nothing
|
||||
|
||||
@ -208,15 +218,16 @@ convExtRel
|
||||
-> RelName
|
||||
-> Maybe RelName
|
||||
-> SelectQExt
|
||||
-> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> m (Either ObjSel ArrSel)
|
||||
convExtRel fieldInfoMap relName mAlias selQ prepValBuilder = do
|
||||
convExtRel fieldInfoMap relName mAlias selQ sessVarBldr prepValBldr = do
|
||||
-- Point to the name key
|
||||
relInfo <- withPathK "name" $
|
||||
askRelType fieldInfoMap relName pgWhenRelErr
|
||||
let (RelInfo _ relTy colMapping relTab _) = relInfo
|
||||
(relCIM, relSPI) <- fetchRelDet relName relTab
|
||||
annSel <- convSelectQ relCIM relSPI selQ prepValBuilder
|
||||
annSel <- convSelectQ relCIM relSPI selQ sessVarBldr prepValBldr
|
||||
case relTy of
|
||||
ObjRel -> do
|
||||
when misused $ throw400 UnexpectedPayload objRelMisuseMsg
|
||||
@ -249,7 +260,7 @@ partAnnFlds flds =
|
||||
FExp _ -> Nothing
|
||||
|
||||
getSelectDeps
|
||||
:: AnnSel
|
||||
:: AnnSimpleSel
|
||||
-> [SchemaDependency]
|
||||
getSelectDeps (AnnSelG flds tabFrm _ tableArgs _) =
|
||||
mkParentDep tn
|
||||
@ -280,49 +291,59 @@ getSelectDeps (AnnSelG flds tabFrm _ tableArgs _) =
|
||||
|
||||
convSelectQuery
|
||||
:: (UserInfoM m, QErrM m, CacheRM m, HasSQLGenCtx m)
|
||||
=> (PGColType -> Value -> m S.SQLExp)
|
||||
=> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> SelectQuery
|
||||
-> m AnnSel
|
||||
convSelectQuery prepArgBuilder (DMLQuery qt selQ) = do
|
||||
-> m AnnSimpleSel
|
||||
convSelectQuery sessVarBldr prepArgBuilder (DMLQuery qt selQ) = do
|
||||
tabInfo <- withPathK "table" $ askTabInfo qt
|
||||
selPermInfo <- askSelPermInfo tabInfo
|
||||
extSelQ <- resolveStar (tiFieldInfoMap tabInfo) selPermInfo selQ
|
||||
validateHeaders $ spiRequiredHeaders selPermInfo
|
||||
convSelectQ (tiFieldInfoMap tabInfo) selPermInfo extSelQ prepArgBuilder
|
||||
convSelectQ (tiFieldInfoMap tabInfo) selPermInfo
|
||||
extSelQ sessVarBldr prepArgBuilder
|
||||
|
||||
funcQueryTx
|
||||
:: S.FromItem -> QualifiedFunction -> QualifiedTable
|
||||
-> TablePerm -> TableArgs -> Bool
|
||||
-> (Either TableAggFlds AnnFlds, DS.Seq Q.PrepArg)
|
||||
-> Q.TxE QErr EncJSON
|
||||
funcQueryTx frmItem fn tn tabPerm tabArgs strfyNum (eSelFlds, p) =
|
||||
encJFromBS . runIdentity . Q.getRow
|
||||
<$> Q.rawQE dmlTxErrorHandler (Q.fromBuilder sqlBuilder) (toList p) True
|
||||
where
|
||||
sqlBuilder = toSQL $
|
||||
mkFuncSelectWith fn tn tabPerm tabArgs strfyNum eSelFlds frmItem
|
||||
mkFuncSelectSimple
|
||||
:: AnnFnSelSimple
|
||||
-> Q.Query
|
||||
mkFuncSelectSimple annFnSel =
|
||||
Q.fromBuilder $ toSQL $
|
||||
mkFuncSelectWith (mkSQLSelect False) annFnSel
|
||||
|
||||
selectAggP2 :: (AnnAggSel, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
selectAggP2 (sel, p) =
|
||||
encJFromBS . runIdentity . Q.getRow
|
||||
<$> Q.rawQE dmlTxErrorHandler (Q.fromBuilder selectSQL) (toList p) True
|
||||
where
|
||||
selectSQL = toSQL $ mkAggSelect sel
|
||||
mkFuncSelectAgg
|
||||
:: AnnFnSelAgg
|
||||
-> Q.Query
|
||||
mkFuncSelectAgg annFnSel =
|
||||
Q.fromBuilder $ toSQL $
|
||||
mkFuncSelectWith mkAggSelect annFnSel
|
||||
|
||||
selectP2 :: Bool -> (AnnSel, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
selectP2 :: Bool -> (AnnSimpleSel, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
selectP2 asSingleObject (sel, p) =
|
||||
encJFromBS . runIdentity . Q.getRow
|
||||
<$> Q.rawQE dmlTxErrorHandler (Q.fromBuilder selectSQL) (toList p) True
|
||||
where
|
||||
selectSQL = toSQL $ mkSQLSelect asSingleObject sel
|
||||
|
||||
selectQuerySQL :: Bool -> AnnSimpleSel -> Q.Query
|
||||
selectQuerySQL asSingleObject sel =
|
||||
Q.fromBuilder $ toSQL $ mkSQLSelect asSingleObject sel
|
||||
|
||||
selectAggQuerySQL :: AnnAggSel -> Q.Query
|
||||
selectAggQuerySQL =
|
||||
Q.fromBuilder . toSQL . mkAggSelect
|
||||
|
||||
asSingleRowJsonResp :: Q.Query -> [Q.PrepArg] -> Q.TxE QErr EncJSON
|
||||
asSingleRowJsonResp query args =
|
||||
encJFromBS . runIdentity . Q.getRow
|
||||
<$> Q.rawQE dmlTxErrorHandler query args True
|
||||
|
||||
phaseOne
|
||||
:: (QErrM m, UserInfoM m, CacheRM m, HasSQLGenCtx m)
|
||||
=> SelectQuery -> m (AnnSel, DS.Seq Q.PrepArg)
|
||||
=> SelectQuery -> m (AnnSimpleSel, DS.Seq Q.PrepArg)
|
||||
phaseOne =
|
||||
liftDMLP1 . convSelectQuery binRHSBuilder
|
||||
liftDMLP1 . convSelectQuery sessVarFromCurrentSetting binRHSBuilder
|
||||
|
||||
phaseTwo :: (MonadTx m) => (AnnSel, DS.Seq Q.PrepArg) -> m EncJSON
|
||||
phaseTwo :: (MonadTx m) => (AnnSimpleSel, DS.Seq Q.PrepArg) -> m EncJSON
|
||||
phaseTwo =
|
||||
liftTx . selectP2 False
|
||||
|
||||
|
@ -527,7 +527,7 @@ mkBaseNode pfx fldAls annSelFlds tableFrom tablePerm tableArgs strfyNum =
|
||||
FArr ar -> Just (f, ar)
|
||||
_ -> Nothing
|
||||
|
||||
annSelToBaseNode :: Iden -> FieldName -> AnnSel -> BaseNode
|
||||
annSelToBaseNode :: Iden -> FieldName -> AnnSimpleSel -> BaseNode
|
||||
annSelToBaseNode pfx fldAls annSel =
|
||||
mkBaseNode pfx fldAls (TAFNodes selFlds) tabFrm tabPerm tabArgs strfyNum
|
||||
where
|
||||
@ -613,7 +613,7 @@ mkAggSelect annAggSel =
|
||||
ArrNode extr _ bn =
|
||||
aggSelToArrNode (Iden "root") (FieldName "root") aggSel
|
||||
|
||||
mkSQLSelect :: Bool -> AnnSel -> S.Select
|
||||
mkSQLSelect :: Bool -> AnnSimpleSel -> S.Select
|
||||
mkSQLSelect isSingleObject annSel =
|
||||
prefixNumToAliases $ arrNodeToSelect baseNode extrs $ S.BELit True
|
||||
where
|
||||
@ -623,26 +623,25 @@ mkSQLSelect isSingleObject annSel =
|
||||
rootFldAls = S.Alias $ toIden rootFldName
|
||||
|
||||
mkFuncSelectWith
|
||||
:: QualifiedFunction -> QualifiedTable
|
||||
-> TablePerm -> TableArgs -> Bool
|
||||
-> Either TableAggFlds AnnFlds -> S.FromItem -> S.SelectWith
|
||||
mkFuncSelectWith qf tn tabPerm tabArgs strfyNum eSelFlds frmItem = selWith
|
||||
:: (AnnSelG a S.SQLExp -> S.Select)
|
||||
-> AnnFnSelG (AnnSelG a S.SQLExp) S.SQLExp
|
||||
-> S.SelectWith
|
||||
mkFuncSelectWith f annFn =
|
||||
S.SelectWith [(funcAls, S.CTESelect funcSel)] $
|
||||
-- we'll need to modify the table from of the underlying
|
||||
-- select to the alias of the select from function
|
||||
f annSel { _asnFrom = newTabFrom }
|
||||
where
|
||||
AnnFnSel qf fnArgs annSel = annFn
|
||||
|
||||
-- SELECT * FROM function_name(args)
|
||||
funcSel = S.mkSelect { S.selFrom = Just $ S.FromExp [frmItem]
|
||||
, S.selExtr = [S.Extractor S.SEStar Nothing]
|
||||
}
|
||||
frmItem = S.mkFuncFromItem qf fnArgs
|
||||
|
||||
mainSel = case eSelFlds of
|
||||
Left aggFlds -> mkAggSelect $
|
||||
AnnSelG aggFlds tabFrom tabPerm tabArgs strfyNum
|
||||
Right annFlds -> mkSQLSelect False $
|
||||
AnnSelG annFlds tabFrom tabPerm tabArgs strfyNum
|
||||
|
||||
tabFrom = TableFrom tn $ Just $ toIden funcAls
|
||||
newTabFrom = (_asnFrom annSel) {_tfIden = Just $ toIden funcAls}
|
||||
|
||||
QualifiedObject sn fn = qf
|
||||
funcAls = S.Alias $ Iden $
|
||||
getSchemaTxt sn <> "_" <> getFunctionTxt fn <> "__result"
|
||||
|
||||
selWith = S.SelectWith [(funcAls, S.CTESelect funcSel)] mainSel
|
||||
|
@ -48,33 +48,71 @@ data AnnAggOrdBy
|
||||
| AAOOp !T.Text !PGCol
|
||||
deriving (Show, Eq)
|
||||
|
||||
data AnnObCol
|
||||
data AnnObColG v
|
||||
= AOCPG !PGColInfo
|
||||
| AOCObj !RelInfo !AnnBoolExpSQL !AnnObCol
|
||||
| AOCAgg !RelInfo !AnnBoolExpSQL !AnnAggOrdBy
|
||||
| AOCObj !RelInfo !(AnnBoolExp v) !(AnnObColG v)
|
||||
| AOCAgg !RelInfo !(AnnBoolExp v) !AnnAggOrdBy
|
||||
deriving (Show, Eq)
|
||||
|
||||
type AnnOrderByItem = OrderByItemG AnnObCol
|
||||
traverseAnnObCol
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> AnnObColG a -> f (AnnObColG b)
|
||||
traverseAnnObCol f = \case
|
||||
AOCPG pgColInfo -> pure $ AOCPG pgColInfo
|
||||
AOCObj relInfo annBoolExp annObCol ->
|
||||
AOCObj relInfo
|
||||
<$> traverseAnnBoolExp f annBoolExp
|
||||
<*> traverseAnnObCol f annObCol
|
||||
AOCAgg relInfo annBoolExp annAggOb ->
|
||||
AOCAgg relInfo
|
||||
<$> traverseAnnBoolExp f annBoolExp
|
||||
<*> pure annAggOb
|
||||
|
||||
type AnnObCol = AnnObColG S.SQLExp
|
||||
|
||||
type AnnOrderByItemG v = OrderByItemG (AnnObColG v)
|
||||
|
||||
traverseAnnOrderByItem
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> AnnOrderByItemG a -> f (AnnOrderByItemG b)
|
||||
traverseAnnOrderByItem f =
|
||||
traverse (traverseAnnObCol f)
|
||||
|
||||
type AnnOrderByItem = AnnOrderByItemG S.SQLExp
|
||||
|
||||
data AnnRelG a
|
||||
= AnnRelG
|
||||
{ aarName :: !RelName -- Relationship name
|
||||
, aarMapping :: ![(PGCol, PGCol)] -- Column of left table to join with
|
||||
, aarAnnSel :: !a -- Current table. Almost ~ to SQL Select
|
||||
} deriving (Show, Eq)
|
||||
} deriving (Show, Eq, Functor, Foldable, Traversable)
|
||||
|
||||
type ObjSel = AnnRelG AnnSel
|
||||
type ArrRel = AnnRelG AnnSel
|
||||
type ArrRelAgg = AnnRelG AnnAggSel
|
||||
type ObjSelG v = AnnRelG (AnnSimpleSelG v)
|
||||
type ObjSel = ObjSelG S.SQLExp
|
||||
type ArrRelG v = AnnRelG (AnnSimpleSelG v)
|
||||
type ArrRelAggG v = AnnRelG (AnnAggSelG v)
|
||||
|
||||
type ArrRelAgg = ArrRelAggG S.SQLExp
|
||||
|
||||
type Fields a = [(FieldName, a)]
|
||||
|
||||
data ArrSel
|
||||
= ASSimple !ArrRel
|
||||
| ASAgg !ArrRelAgg
|
||||
data ArrSelG v
|
||||
= ASSimple !(ArrRelG v)
|
||||
| ASAgg !(ArrRelAggG v)
|
||||
deriving (Show, Eq)
|
||||
|
||||
type ArrSelFlds = Fields ArrSel
|
||||
traverseArrSel
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> ArrSelG a
|
||||
-> f (ArrSelG b)
|
||||
traverseArrSel f = \case
|
||||
ASSimple arrRel -> ASSimple <$> traverse (traverseAnnSimpleSel f) arrRel
|
||||
ASAgg arrRelAgg -> ASAgg <$> traverse (traverseAnnAggSel f) arrRelAgg
|
||||
|
||||
type ArrSel = ArrSelG S.SQLExp
|
||||
|
||||
type ArrSelFldsG v = Fields (ArrSelG v)
|
||||
|
||||
data ColOp
|
||||
= ColOp
|
||||
@ -82,23 +120,48 @@ data ColOp
|
||||
, _colExp :: S.SQLExp
|
||||
} deriving (Show, Eq)
|
||||
|
||||
data AnnFld
|
||||
data AnnFldG v
|
||||
= FCol !PGColInfo !(Maybe ColOp)
|
||||
| FObj !ObjSel
|
||||
| FArr !ArrSel
|
||||
| FObj !(ObjSelG v)
|
||||
| FArr !(ArrSelG v)
|
||||
| FExp !T.Text
|
||||
deriving (Show, Eq)
|
||||
|
||||
data TableArgs
|
||||
traverseAnnFld
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> AnnFldG a -> f (AnnFldG b)
|
||||
traverseAnnFld f = \case
|
||||
FCol pgColInfo colOpM -> pure $ FCol pgColInfo colOpM
|
||||
FObj sel -> FObj <$> traverse (traverseAnnSimpleSel f) sel
|
||||
FArr sel -> FArr <$> traverseArrSel f sel
|
||||
FExp t -> FExp <$> pure t
|
||||
|
||||
type AnnFld = AnnFldG S.SQLExp
|
||||
|
||||
data TableArgsG v
|
||||
= TableArgs
|
||||
{ _taWhere :: !(Maybe AnnBoolExpSQL)
|
||||
, _taOrderBy :: !(Maybe (NE.NonEmpty AnnOrderByItem))
|
||||
{ _taWhere :: !(Maybe (AnnBoolExp v))
|
||||
, _taOrderBy :: !(Maybe (NE.NonEmpty (AnnOrderByItemG v)))
|
||||
, _taLimit :: !(Maybe Int)
|
||||
, _taOffset :: !(Maybe S.SQLExp)
|
||||
, _taDistCols :: !(Maybe (NE.NonEmpty PGCol))
|
||||
} deriving (Show, Eq)
|
||||
|
||||
noTableArgs :: TableArgs
|
||||
traverseTableArgs
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> TableArgsG a -> f (TableArgsG b)
|
||||
traverseTableArgs f (TableArgs wh ordBy lmt ofst distCols) =
|
||||
TableArgs
|
||||
<$> traverse (traverseAnnBoolExp f) wh
|
||||
-- traversing through maybe -> nonempty -> annorderbyitem
|
||||
<*> traverse (traverse (traverseAnnOrderByItem f)) ordBy
|
||||
<*> pure lmt
|
||||
<*> pure ofst
|
||||
<*> pure distCols
|
||||
|
||||
type TableArgs = TableArgsG S.SQLExp
|
||||
|
||||
noTableArgs :: TableArgsG v
|
||||
noTableArgs = TableArgs Nothing Nothing Nothing Nothing Nothing
|
||||
|
||||
data PGColFld
|
||||
@ -121,15 +184,28 @@ data AggFld
|
||||
deriving (Show, Eq)
|
||||
|
||||
type AggFlds = Fields AggFld
|
||||
type AnnFlds = Fields AnnFld
|
||||
type AnnFldsG v = Fields (AnnFldG v)
|
||||
|
||||
data TableAggFld
|
||||
type AnnFlds = AnnFldsG S.SQLExp
|
||||
|
||||
data TableAggFldG v
|
||||
= TAFAgg !AggFlds
|
||||
| TAFNodes !AnnFlds
|
||||
| TAFNodes !(AnnFldsG v)
|
||||
| TAFExp !T.Text
|
||||
deriving (Show, Eq)
|
||||
|
||||
type TableAggFlds = Fields TableAggFld
|
||||
traverseTableAggFld
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> TableAggFldG a -> f (TableAggFldG b)
|
||||
traverseTableAggFld f = \case
|
||||
TAFAgg aggFlds -> pure $ TAFAgg aggFlds
|
||||
TAFNodes annFlds ->
|
||||
TAFNodes <$> traverse (traverse (traverseAnnFld f)) annFlds
|
||||
TAFExp t -> pure $ TAFExp t
|
||||
|
||||
type TableAggFld = TableAggFldG S.SQLExp
|
||||
type TableAggFldsG v = Fields (TableAggFldG v)
|
||||
type TableAggFlds = TableAggFldsG S.SQLExp
|
||||
|
||||
data TableFrom
|
||||
= TableFrom
|
||||
@ -137,23 +213,98 @@ data TableFrom
|
||||
, _tfIden :: !(Maybe Iden)
|
||||
} deriving (Show, Eq)
|
||||
|
||||
data TablePerm
|
||||
data TablePermG v
|
||||
= TablePerm
|
||||
{ _tpFilter :: !AnnBoolExpSQL
|
||||
{ _tpFilter :: !(AnnBoolExp v)
|
||||
, _tpLimit :: !(Maybe Int)
|
||||
} deriving (Eq, Show)
|
||||
|
||||
data AnnSelG a
|
||||
traverseTablePerm
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> TablePermG a
|
||||
-> f (TablePermG b)
|
||||
traverseTablePerm f (TablePerm boolExp limit) =
|
||||
TablePerm
|
||||
<$> traverseAnnBoolExp f boolExp
|
||||
<*> pure limit
|
||||
|
||||
type TablePerm = TablePermG S.SQLExp
|
||||
|
||||
data AnnSelG a v
|
||||
= AnnSelG
|
||||
{ _asnFields :: !a
|
||||
, _asnFrom :: !TableFrom
|
||||
, _asnPerm :: !TablePerm
|
||||
, _asnArgs :: !TableArgs
|
||||
, _asnPerm :: !(TablePermG v)
|
||||
, _asnArgs :: !(TableArgsG v)
|
||||
, _asnStrfyNum :: !Bool
|
||||
} deriving (Show, Eq)
|
||||
|
||||
type AnnSel = AnnSelG AnnFlds
|
||||
type AnnAggSel = AnnSelG TableAggFlds
|
||||
traverseAnnSimpleSel
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnSimpleSelG a -> f (AnnSimpleSelG b)
|
||||
traverseAnnSimpleSel f =
|
||||
traverseAnnSel (traverse (traverse (traverseAnnFld f))) f
|
||||
|
||||
traverseAnnAggSel
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnAggSelG a -> f (AnnAggSelG b)
|
||||
traverseAnnAggSel f =
|
||||
traverseAnnSel (traverse (traverse (traverseTableAggFld f))) f
|
||||
|
||||
traverseAnnSel
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> (v -> f w)
|
||||
-> AnnSelG a v -> f (AnnSelG b w)
|
||||
traverseAnnSel f1 f2 (AnnSelG flds tabFrom perm args strfyNum) =
|
||||
AnnSelG
|
||||
<$> f1 flds
|
||||
<*> pure tabFrom
|
||||
<*> traverseTablePerm f2 perm
|
||||
<*> traverseTableArgs f2 args
|
||||
<*> pure strfyNum
|
||||
|
||||
type AnnSimpleSelG v = AnnSelG (AnnFldsG v) v
|
||||
type AnnSimpleSel = AnnSimpleSelG S.SQLExp
|
||||
|
||||
type AnnAggSelG v = AnnSelG (TableAggFldsG v) v
|
||||
type AnnAggSel = AnnAggSelG S.SQLExp
|
||||
|
||||
data AnnFnSelG s v
|
||||
= AnnFnSel
|
||||
{ _afFn :: !QualifiedFunction
|
||||
, _afFnArgs :: ![v]
|
||||
, _afSelect :: !s
|
||||
} deriving (Show, Eq)
|
||||
|
||||
traverseAnnFnSel
|
||||
:: (Applicative f)
|
||||
=> (a -> f b) -> (v -> f w)
|
||||
-> AnnFnSelG a v -> f (AnnFnSelG b w)
|
||||
traverseAnnFnSel fs fv (AnnFnSel fn fnArgs s) =
|
||||
AnnFnSel fn <$> traverse fv fnArgs <*> fs s
|
||||
|
||||
type AnnFnSelSimpleG v = AnnFnSelG (AnnSimpleSelG v) v
|
||||
type AnnFnSelSimple = AnnFnSelSimpleG S.SQLExp
|
||||
|
||||
traverseAnnFnSimple
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnFnSelSimpleG a -> f (AnnFnSelSimpleG b)
|
||||
traverseAnnFnSimple f =
|
||||
traverseAnnFnSel (traverseAnnSimpleSel f) f
|
||||
|
||||
type AnnFnSelAggG v = AnnFnSelG (AnnAggSelG v) v
|
||||
type AnnFnSelAgg = AnnFnSelAggG S.SQLExp
|
||||
|
||||
traverseAnnFnAgg
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnFnSelAggG a -> f (AnnFnSelAggG b)
|
||||
traverseAnnFnAgg f =
|
||||
traverseAnnFnSel (traverseAnnAggSel f) f
|
||||
|
||||
data BaseNode
|
||||
= BaseNode
|
||||
@ -188,20 +339,24 @@ data OrderByNode
|
||||
| OBNArrNode !S.Alias !ArrNode
|
||||
deriving (Show, Eq)
|
||||
|
||||
data ArrRelCtx
|
||||
data ArrRelCtxG v
|
||||
= ArrRelCtx
|
||||
{ aacFields :: !ArrSelFlds
|
||||
{ aacFields :: !(ArrSelFldsG v)
|
||||
, aacAggOrdBys :: ![RelName]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
emptyArrRelCtx :: ArrRelCtx
|
||||
type ArrRelCtx = ArrRelCtxG S.SQLExp
|
||||
|
||||
emptyArrRelCtx :: ArrRelCtxG a
|
||||
emptyArrRelCtx = ArrRelCtx [] []
|
||||
|
||||
data ArrNodeItem
|
||||
= ANIField !(FieldName, ArrSel)
|
||||
data ArrNodeItemG v
|
||||
= ANIField !(FieldName, ArrSelG v)
|
||||
| ANIAggOrdBy !RelName
|
||||
deriving (Show, Eq)
|
||||
|
||||
type ArrNodeItem = ArrNodeItemG S.SQLExp
|
||||
|
||||
data ObjNode
|
||||
= ObjNode
|
||||
{ _rnRelMapping :: ![(PGCol, PGCol)]
|
||||
|
@ -1,7 +1,9 @@
|
||||
module Hasura.RQL.DML.Update
|
||||
( validateUpdateQueryWith
|
||||
, validateUpdateQuery
|
||||
, UpdateQueryP1(..)
|
||||
, AnnUpdG(..)
|
||||
, traverseAnnUpd
|
||||
, AnnUpd
|
||||
, updateQueryToTx
|
||||
, getUpdateDeps
|
||||
, runUpdate
|
||||
@ -26,18 +28,37 @@ import Hasura.SQL.Types
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Hasura.SQL.DML as S
|
||||
|
||||
data UpdateQueryP1
|
||||
= UpdateQueryP1
|
||||
data AnnUpdG v
|
||||
= AnnUpd
|
||||
{ uqp1Table :: !QualifiedTable
|
||||
, uqp1SetExps :: ![(PGCol, S.SQLExp)]
|
||||
, uqp1Where :: !(AnnBoolExpSQL, AnnBoolExpSQL)
|
||||
, uqp1MutFlds :: !MutFlds
|
||||
, uqp1SetExps :: ![(PGCol, v)]
|
||||
, uqp1Where :: !(AnnBoolExp v, AnnBoolExp v)
|
||||
-- we don't prepare the arguments for returning
|
||||
-- however the session variable can still be
|
||||
-- converted as desired
|
||||
, uqp1MutFlds :: !(MutFldsG v)
|
||||
, uqp1AllCols :: ![PGColInfo]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
traverseAnnUpd
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnUpdG a
|
||||
-> f (AnnUpdG b)
|
||||
traverseAnnUpd f annUpd =
|
||||
AnnUpd tn
|
||||
<$> traverse (traverse f) setExps
|
||||
<*> ((,) <$> traverseAnnBoolExp f whr <*> traverseAnnBoolExp f fltr)
|
||||
<*> traverseMutFlds f mutFlds
|
||||
<*> pure allCols
|
||||
where
|
||||
AnnUpd tn setExps (whr, fltr) mutFlds allCols = annUpd
|
||||
|
||||
type AnnUpd = AnnUpdG S.SQLExp
|
||||
|
||||
mkUpdateCTE
|
||||
:: UpdateQueryP1 -> S.CTE
|
||||
mkUpdateCTE (UpdateQueryP1 tn setExps (permFltr, wc) _ _) =
|
||||
:: AnnUpd -> S.CTE
|
||||
mkUpdateCTE (AnnUpd tn setExps (permFltr, wc) _ _) =
|
||||
S.CTEUpdate update
|
||||
where
|
||||
update = S.SQLUpdate tn setExp Nothing tableFltr $ Just S.returningStar
|
||||
@ -46,9 +67,9 @@ mkUpdateCTE (UpdateQueryP1 tn setExps (permFltr, wc) _ _) =
|
||||
toSQLBoolExp (S.QualTable tn) $ andAnnBoolExps permFltr wc
|
||||
|
||||
getUpdateDeps
|
||||
:: UpdateQueryP1
|
||||
:: AnnUpd
|
||||
-> [SchemaDependency]
|
||||
getUpdateDeps (UpdateQueryP1 tn setExps (_, wc) mutFlds allCols) =
|
||||
getUpdateDeps (AnnUpd tn setExps (_, wc) mutFlds allCols) =
|
||||
mkParentDep tn : colDeps <> allColDeps <> whereDeps <> retDeps
|
||||
where
|
||||
colDeps = map (mkColDep "on_type" tn . fst) setExps
|
||||
@ -120,10 +141,11 @@ convOp fieldInfoMap preSetCols updPerm objs conv =
|
||||
|
||||
validateUpdateQueryWith
|
||||
:: (UserInfoM m, QErrM m, CacheRM m)
|
||||
=> (PGColType -> Value -> m S.SQLExp)
|
||||
=> SessVarBldr m
|
||||
-> (PGColType -> Value -> m S.SQLExp)
|
||||
-> UpdateQuery
|
||||
-> m UpdateQueryP1
|
||||
validateUpdateQueryWith f uq = do
|
||||
-> m AnnUpd
|
||||
validateUpdateQueryWith sessVarBldr prepValBldr uq = do
|
||||
let tableName = uqTable uq
|
||||
tableInfo <- withPathK "table" $ askTabInfo tableName
|
||||
|
||||
@ -148,13 +170,13 @@ validateUpdateQueryWith f uq = do
|
||||
|
||||
-- convert the object to SQL set expression
|
||||
setItems <- withPathK "$set" $
|
||||
convOp fieldInfoMap preSetCols updPerm (M.toList $ uqSet uq) $ convSet f
|
||||
convOp fieldInfoMap preSetCols updPerm (M.toList $ uqSet uq) $ convSet prepValBldr
|
||||
|
||||
incItems <- withPathK "$inc" $
|
||||
convOp fieldInfoMap preSetCols updPerm (M.toList $ uqInc uq) $ convInc f
|
||||
convOp fieldInfoMap preSetCols updPerm (M.toList $ uqInc uq) $ convInc prepValBldr
|
||||
|
||||
mulItems <- withPathK "$mul" $
|
||||
convOp fieldInfoMap preSetCols updPerm (M.toList $ uqMul uq) $ convMul f
|
||||
convOp fieldInfoMap preSetCols updPerm (M.toList $ uqMul uq) $ convMul prepValBldr
|
||||
|
||||
defItems <- withPathK "$default" $
|
||||
convOp fieldInfoMap preSetCols updPerm (zip (uqDefault uq) [()..]) convDefault
|
||||
@ -163,20 +185,26 @@ validateUpdateQueryWith f uq = do
|
||||
mAnnRetCols <- forM mRetCols $ \retCols ->
|
||||
withPathK "returning" $ checkRetCols fieldInfoMap selPerm retCols
|
||||
|
||||
let preSetItems = M.toList preSetObj
|
||||
setExpItems = preSetItems ++ setItems ++ incItems ++ mulItems ++ defItems
|
||||
resolvedPreSetItems <- M.toList <$>
|
||||
mapM (convPartialSQLExp sessVarBldr) preSetObj
|
||||
|
||||
let setExpItems = resolvedPreSetItems ++ setItems ++ incItems ++
|
||||
mulItems ++ defItems
|
||||
|
||||
when (null setExpItems) $
|
||||
throw400 UnexpectedPayload "atleast one of $set, $inc, $mul has to be present"
|
||||
|
||||
-- convert the where clause
|
||||
annSQLBoolExp <- withPathK "where" $
|
||||
convBoolExp' fieldInfoMap selPerm (uqWhere uq) f
|
||||
convBoolExp fieldInfoMap selPerm (uqWhere uq) sessVarBldr prepValBldr
|
||||
|
||||
return $ UpdateQueryP1
|
||||
resolvedUpdFltr <- convAnnBoolExpPartialSQL sessVarBldr $
|
||||
upiFilter updPerm
|
||||
|
||||
return $ AnnUpd
|
||||
tableName
|
||||
setExpItems
|
||||
(upiFilter updPerm, annSQLBoolExp)
|
||||
(resolvedUpdFltr, annSQLBoolExp)
|
||||
(mkDefaultMutFlds mAnnRetCols)
|
||||
allCols
|
||||
where
|
||||
@ -188,12 +216,12 @@ validateUpdateQueryWith f uq = do
|
||||
|
||||
validateUpdateQuery
|
||||
:: (QErrM m, UserInfoM m, CacheRM m, HasSQLGenCtx m)
|
||||
=> UpdateQuery -> m (UpdateQueryP1, DS.Seq Q.PrepArg)
|
||||
=> UpdateQuery -> m (AnnUpd, DS.Seq Q.PrepArg)
|
||||
validateUpdateQuery =
|
||||
liftDMLP1 . validateUpdateQueryWith binRHSBuilder
|
||||
liftDMLP1 . validateUpdateQueryWith sessVarFromCurrentSetting binRHSBuilder
|
||||
|
||||
updateQueryToTx
|
||||
:: Bool -> (UpdateQueryP1, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
:: Bool -> (AnnUpd, DS.Seq Q.PrepArg) -> Q.TxE QErr EncJSON
|
||||
updateQueryToTx strfyNum (u, p) =
|
||||
runMutation $ Mutation (uqp1Table u) (updateCTE, p)
|
||||
(uqp1MutFlds u) (uqp1AllCols u) strfyNum
|
||||
|
@ -6,10 +6,6 @@ module Hasura.RQL.Types
|
||||
, liftP1WithQCtx
|
||||
, MonadTx(..)
|
||||
|
||||
, LazyTx
|
||||
, runLazyTx
|
||||
, withUserInfo
|
||||
|
||||
, UserInfoM(..)
|
||||
, successMsg
|
||||
|
||||
@ -36,7 +32,6 @@ module Hasura.RQL.Types
|
||||
, askQTemplateInfo
|
||||
|
||||
, adminOnly
|
||||
, defaultTxErrorHandler
|
||||
|
||||
, HeaderObj
|
||||
|
||||
@ -44,6 +39,7 @@ module Hasura.RQL.Types
|
||||
, module R
|
||||
) where
|
||||
|
||||
import Hasura.Db as R
|
||||
import Hasura.EncJSON
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types.BoolExp as R
|
||||
@ -59,14 +55,8 @@ import Hasura.SQL.Types
|
||||
|
||||
import qualified Hasura.GraphQL.Context as GC
|
||||
|
||||
import qualified Database.PG.Query as Q
|
||||
|
||||
import Data.Aeson
|
||||
|
||||
import qualified Data.Aeson.Text as AT
|
||||
import qualified Data.HashMap.Strict as M
|
||||
import qualified Data.Text as T
|
||||
import qualified Data.Text.Lazy as LT
|
||||
import qualified Network.HTTP.Client as HTTP
|
||||
|
||||
getFieldInfoMap
|
||||
@ -155,86 +145,6 @@ newtype SQLGenCtx
|
||||
class (Monad m) => HasSQLGenCtx m where
|
||||
askSQLGenCtx :: m SQLGenCtx
|
||||
|
||||
class (MonadError QErr m) => MonadTx m where
|
||||
liftTx :: Q.TxE QErr a -> m a
|
||||
|
||||
instance (MonadTx m) => MonadTx (StateT s m) where
|
||||
liftTx = lift . liftTx
|
||||
|
||||
instance (MonadTx m) => MonadTx (ReaderT s m) where
|
||||
liftTx = lift . liftTx
|
||||
|
||||
data LazyTx e a
|
||||
= LTErr !e
|
||||
| LTNoTx !a
|
||||
| LTTx !(Q.TxE e a)
|
||||
|
||||
lazyTxToQTx :: LazyTx e a -> Q.TxE e a
|
||||
lazyTxToQTx = \case
|
||||
LTErr e -> throwError e
|
||||
LTNoTx r -> return r
|
||||
LTTx tx -> tx
|
||||
|
||||
runLazyTx
|
||||
:: Q.PGPool -> Q.TxIsolation
|
||||
-> LazyTx QErr a -> ExceptT QErr IO a
|
||||
runLazyTx pgPool txIso = \case
|
||||
LTErr e -> throwError e
|
||||
LTNoTx a -> return a
|
||||
LTTx tx -> Q.runTx pgPool (txIso, Nothing) tx
|
||||
|
||||
setHeadersTx :: UserVars -> Q.TxE QErr ()
|
||||
setHeadersTx uVars =
|
||||
Q.unitQE defaultTxErrorHandler setSess () False
|
||||
where
|
||||
toStrictText = LT.toStrict . AT.encodeToLazyText
|
||||
setSess = Q.fromText $
|
||||
"SET LOCAL \"hasura.user\" = " <>
|
||||
pgFmtLit (toStrictText uVars)
|
||||
|
||||
withUserInfo :: UserInfo -> LazyTx QErr a -> LazyTx QErr a
|
||||
withUserInfo uInfo = \case
|
||||
LTErr e -> LTErr e
|
||||
LTNoTx a -> LTNoTx a
|
||||
LTTx tx -> LTTx $ setHeadersTx (userVars uInfo) >> tx
|
||||
|
||||
instance Functor (LazyTx e) where
|
||||
fmap f = \case
|
||||
LTErr e -> LTErr e
|
||||
LTNoTx a -> LTNoTx $ f a
|
||||
LTTx tx -> LTTx $ fmap f tx
|
||||
|
||||
instance Applicative (LazyTx e) where
|
||||
pure = LTNoTx
|
||||
|
||||
LTErr e <*> _ = LTErr e
|
||||
LTNoTx f <*> r = fmap f r
|
||||
LTTx _ <*> LTErr e = LTErr e
|
||||
LTTx txf <*> LTNoTx a = LTTx $ txf <*> pure a
|
||||
LTTx txf <*> LTTx tx = LTTx $ txf <*> tx
|
||||
|
||||
instance Monad (LazyTx e) where
|
||||
LTErr e >>= _ = LTErr e
|
||||
LTNoTx a >>= f = f a
|
||||
LTTx txa >>= f =
|
||||
LTTx $ txa >>= lazyTxToQTx . f
|
||||
|
||||
instance MonadError e (LazyTx e) where
|
||||
throwError = LTErr
|
||||
LTErr e `catchError` f = f e
|
||||
LTNoTx a `catchError` _ = LTNoTx a
|
||||
LTTx txe `catchError` f =
|
||||
LTTx $ txe `catchError` (lazyTxToQTx . f)
|
||||
|
||||
instance MonadTx (LazyTx QErr) where
|
||||
liftTx = LTTx
|
||||
|
||||
instance MonadTx (Q.TxE QErr) where
|
||||
liftTx = id
|
||||
|
||||
instance MonadIO (LazyTx QErr) where
|
||||
liftIO = LTTx . liftIO
|
||||
|
||||
type ER e r = ExceptT e (Reader r)
|
||||
type P1 = ER QErr QCtx
|
||||
|
||||
@ -347,11 +257,6 @@ adminOnly = do
|
||||
where
|
||||
errMsg = "restricted access : admin only"
|
||||
|
||||
defaultTxErrorHandler :: Q.PGTxErr -> QErr
|
||||
defaultTxErrorHandler txe =
|
||||
let e = err500 PostgresError "postgres query error"
|
||||
in e {qeInternal = Just $ toJSON txe}
|
||||
|
||||
successMsg :: EncJSON
|
||||
successMsg = "{\"message\":\"success\"}"
|
||||
|
||||
|
@ -10,16 +10,25 @@ module Hasura.RQL.Types.BoolExp
|
||||
|
||||
, AnnBoolExpFld(..)
|
||||
, AnnBoolExp
|
||||
, traverseAnnBoolExp
|
||||
, fmapAnnBoolExp
|
||||
, annBoolExpTrue
|
||||
, andAnnBoolExps
|
||||
|
||||
, AnnBoolExpFldSQL
|
||||
, AnnBoolExpSQL
|
||||
, PartialSQLExp(..)
|
||||
, AnnBoolExpFldPartialSQL
|
||||
, AnnBoolExpPartialSQL
|
||||
|
||||
, PreSetCols
|
||||
, PreSetColsPartial
|
||||
, foldBoolExp
|
||||
) where
|
||||
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types.Common
|
||||
import Hasura.RQL.Types.Permission
|
||||
import qualified Hasura.SQL.DML as S
|
||||
import Hasura.SQL.Types
|
||||
|
||||
@ -214,6 +223,25 @@ data AnnBoolExpFld a
|
||||
type AnnBoolExp a
|
||||
= GBoolExp (AnnBoolExpFld a)
|
||||
|
||||
traverseAnnBoolExp
|
||||
:: (Applicative f)
|
||||
=> (a -> f b)
|
||||
-> AnnBoolExp a
|
||||
-> f (AnnBoolExp b)
|
||||
traverseAnnBoolExp f =
|
||||
traverse $ \case
|
||||
AVCol pgColInfo opExps ->
|
||||
AVCol pgColInfo <$> traverse (traverse f) opExps
|
||||
AVRel relInfo annBoolExp ->
|
||||
AVRel relInfo <$> traverseAnnBoolExp f annBoolExp
|
||||
|
||||
fmapAnnBoolExp
|
||||
:: (a -> b)
|
||||
-> AnnBoolExp a
|
||||
-> AnnBoolExp b
|
||||
fmapAnnBoolExp f =
|
||||
runIdentity . traverseAnnBoolExp (pure . f)
|
||||
|
||||
annBoolExpTrue :: AnnBoolExp a
|
||||
annBoolExpTrue = gBoolExpTrue
|
||||
|
||||
@ -223,8 +251,24 @@ andAnnBoolExps l r =
|
||||
|
||||
type AnnBoolExpFldSQL = AnnBoolExpFld S.SQLExp
|
||||
type AnnBoolExpSQL = AnnBoolExp S.SQLExp
|
||||
type AnnBoolExpFldPartialSQL = AnnBoolExpFld PartialSQLExp
|
||||
type AnnBoolExpPartialSQL = AnnBoolExp PartialSQLExp
|
||||
|
||||
instance ToJSON AnnBoolExpSQL where
|
||||
type PreSetColsPartial = M.HashMap PGCol PartialSQLExp
|
||||
type PreSetCols = M.HashMap PGCol S.SQLExp
|
||||
|
||||
-- doesn't resolve the session variable
|
||||
data PartialSQLExp
|
||||
= PSESessVar !PGColType !SessVar
|
||||
| PSESQLExp !S.SQLExp
|
||||
deriving (Show, Eq)
|
||||
|
||||
instance ToJSON PartialSQLExp where
|
||||
toJSON = \case
|
||||
PSESessVar colTy sessVar -> toJSON (colTy, sessVar)
|
||||
PSESQLExp e -> toJSON $ toSQLTxt e
|
||||
|
||||
instance ToJSON AnnBoolExpPartialSQL where
|
||||
toJSON = gBoolExpToJSON f
|
||||
where
|
||||
f annFld = case annFld of
|
||||
@ -236,6 +280,6 @@ instance ToJSON AnnBoolExpSQL where
|
||||
( getRelTxt $ riName ri
|
||||
, toJSON (ri, toJSON relBoolExp)
|
||||
)
|
||||
opExpSToJSON :: OpExpG S.SQLExp -> Value
|
||||
opExpSToJSON :: OpExpG PartialSQLExp -> Value
|
||||
opExpSToJSON =
|
||||
object . pure . opExpToJPair (toJSON . toSQLTxt)
|
||||
object . pure . opExpToJPair toJSON
|
||||
|
@ -15,12 +15,10 @@ module Hasura.RQL.Types.Common
|
||||
, ToAesonPairs(..)
|
||||
, WithTable(..)
|
||||
, ColVals
|
||||
, PreSetCols
|
||||
, MutateResp(..)
|
||||
) where
|
||||
|
||||
import Hasura.Prelude
|
||||
import qualified Hasura.SQL.DML as S
|
||||
import Hasura.SQL.Types
|
||||
|
||||
import Data.Aeson
|
||||
@ -140,7 +138,6 @@ instance (ToAesonPairs a) => ToJSON (WithTable a) where
|
||||
object $ ("table" .= tn):toAesonPairs rel
|
||||
|
||||
type ColVals = HM.HashMap PGCol Value
|
||||
type PreSetCols = HM.HashMap PGCol S.SQLExp
|
||||
|
||||
data MutateResp
|
||||
= MutateResp
|
||||
|
@ -10,11 +10,13 @@ module Hasura.RQL.Types.Error
|
||||
, err404
|
||||
, err401
|
||||
, err500
|
||||
, internalError
|
||||
|
||||
, QErrM
|
||||
, throw400
|
||||
, throw404
|
||||
, throw500
|
||||
, throw500WithDetail
|
||||
, throw401
|
||||
|
||||
-- Aeson helpers
|
||||
@ -214,7 +216,14 @@ throw401 :: (QErrM m) => T.Text -> m a
|
||||
throw401 t = throwError $ err401 AccessDenied t
|
||||
|
||||
throw500 :: (QErrM m) => T.Text -> m a
|
||||
throw500 t = throwError $ err500 Unexpected t
|
||||
throw500 t = throwError $ internalError t
|
||||
|
||||
internalError :: Text -> QErr
|
||||
internalError = err500 Unexpected
|
||||
|
||||
throw500WithDetail :: (QErrM m) => T.Text -> Value -> m a
|
||||
throw500WithDetail t detail =
|
||||
throwError $ (err500 Unexpected t) {qeInternal = Just detail}
|
||||
|
||||
modifyQErr :: (QErrM m)
|
||||
=> (QErr -> QErr) -> m a -> m a
|
||||
|
@ -1,6 +1,8 @@
|
||||
module Hasura.RQL.Types.Permission
|
||||
( RoleName(..)
|
||||
, UserId(..)
|
||||
|
||||
, SessVar
|
||||
, SessVarVal
|
||||
|
||||
, UserVars
|
||||
, mkUserVars
|
||||
@ -51,11 +53,11 @@ adminRole = RoleName "admin"
|
||||
isAdmin :: RoleName -> Bool
|
||||
isAdmin = (adminRole ==)
|
||||
|
||||
newtype UserId = UserId { getUserId :: Word64 }
|
||||
deriving (Show, Eq, FromJSON, ToJSON)
|
||||
type SessVar = Text
|
||||
type SessVarVal = Text
|
||||
|
||||
newtype UserVars
|
||||
= UserVars { unUserVars :: Map.HashMap T.Text T.Text }
|
||||
= UserVars { unUserVars :: Map.HashMap SessVar SessVarVal}
|
||||
deriving (Show, Eq, FromJSON, ToJSON, Hashable)
|
||||
|
||||
isUserVar :: T.Text -> Bool
|
||||
@ -65,9 +67,9 @@ roleFromVars :: UserVars -> Maybe RoleName
|
||||
roleFromVars =
|
||||
fmap RoleName . getVarVal userRoleHeader
|
||||
|
||||
getVarVal :: Text -> UserVars -> Maybe Text
|
||||
getVarVal :: SessVar -> UserVars -> Maybe SessVarVal
|
||||
getVarVal k =
|
||||
Map.lookup k . unUserVars
|
||||
Map.lookup (T.toLower k) . unUserVars
|
||||
|
||||
getVarNames :: UserVars -> [T.Text]
|
||||
getVarNames =
|
||||
|
@ -4,6 +4,9 @@
|
||||
module Hasura.RQL.Types.SchemaCache
|
||||
( TableCache
|
||||
, SchemaCache(..)
|
||||
, SchemaCacheVer
|
||||
, initSchemaCacheVer
|
||||
, incSchemaCacheVer
|
||||
, emptySchemaCache
|
||||
, TableInfo(..)
|
||||
, TableConstraint(..)
|
||||
@ -63,7 +66,7 @@ module Hasura.RQL.Types.SchemaCache
|
||||
, DelPermInfo(..)
|
||||
, addPermToCache
|
||||
, delPermFromCache
|
||||
, PreSetCols
|
||||
, PreSetColsPartial
|
||||
|
||||
, QueryTemplateInfo(..)
|
||||
, addQTemplateToCache
|
||||
@ -198,8 +201,8 @@ data InsPermInfo
|
||||
= InsPermInfo
|
||||
{ ipiCols :: !(HS.HashSet PGCol)
|
||||
, ipiView :: !QualifiedTable
|
||||
, ipiCheck :: !AnnBoolExpSQL
|
||||
, ipiSet :: !PreSetCols
|
||||
, ipiCheck :: !AnnBoolExpPartialSQL
|
||||
, ipiSet :: !PreSetColsPartial
|
||||
, ipiRequiredHeaders :: ![T.Text]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -209,7 +212,7 @@ data SelPermInfo
|
||||
= SelPermInfo
|
||||
{ spiCols :: !(HS.HashSet PGCol)
|
||||
, spiTable :: !QualifiedTable
|
||||
, spiFilter :: !AnnBoolExpSQL
|
||||
, spiFilter :: !AnnBoolExpPartialSQL
|
||||
, spiLimit :: !(Maybe Int)
|
||||
, spiAllowAgg :: !Bool
|
||||
, spiRequiredHeaders :: ![T.Text]
|
||||
@ -221,8 +224,8 @@ data UpdPermInfo
|
||||
= UpdPermInfo
|
||||
{ upiCols :: !(HS.HashSet PGCol)
|
||||
, upiTable :: !QualifiedTable
|
||||
, upiFilter :: !AnnBoolExpSQL
|
||||
, upiSet :: !PreSetCols
|
||||
, upiFilter :: !AnnBoolExpPartialSQL
|
||||
, upiSet :: !PreSetColsPartial
|
||||
, upiRequiredHeaders :: ![T.Text]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -231,7 +234,7 @@ $(deriveToJSON (aesonDrop 3 snakeCase) ''UpdPermInfo)
|
||||
data DelPermInfo
|
||||
= DelPermInfo
|
||||
{ dpiTable :: !QualifiedTable
|
||||
, dpiFilter :: !AnnBoolExpSQL
|
||||
, dpiFilter :: !AnnBoolExpPartialSQL
|
||||
, dpiRequiredHeaders :: ![T.Text]
|
||||
} deriving (Show, Eq)
|
||||
|
||||
@ -421,6 +424,17 @@ removeFromDepMap :: SchemaObjId -> DepMap -> DepMap
|
||||
removeFromDepMap =
|
||||
M.delete
|
||||
|
||||
newtype SchemaCacheVer
|
||||
= SchemaCacheVer { unSchemaCacheVer :: Word64 }
|
||||
deriving (Show, Eq, Hashable, ToJSON, FromJSON)
|
||||
|
||||
initSchemaCacheVer :: SchemaCacheVer
|
||||
initSchemaCacheVer = SchemaCacheVer 0
|
||||
|
||||
incSchemaCacheVer :: SchemaCacheVer -> SchemaCacheVer
|
||||
incSchemaCacheVer (SchemaCacheVer prev) =
|
||||
SchemaCacheVer $ prev + 1
|
||||
|
||||
data SchemaCache
|
||||
= SchemaCache
|
||||
{ scTables :: !TableCache
|
||||
|
@ -282,6 +282,9 @@ data SQLExp
|
||||
| SECount !CountType
|
||||
deriving (Show, Eq)
|
||||
|
||||
withTyAnn :: PGColType -> SQLExp -> SQLExp
|
||||
withTyAnn colTy v = SETyAnn v $ AnnType $ T.pack $ show colTy
|
||||
|
||||
instance J.ToJSON SQLExp where
|
||||
toJSON = J.toJSON . toSQLTxt
|
||||
|
||||
@ -342,10 +345,6 @@ intToSQLExp :: Int -> SQLExp
|
||||
intToSQLExp =
|
||||
SEUnsafe . T.pack . show
|
||||
|
||||
annotateExp :: SQLExp -> PGColType -> SQLExp
|
||||
annotateExp sqlExp =
|
||||
SETyAnn sqlExp . AnnType . T.pack . show
|
||||
|
||||
data Extractor = Extractor !SQLExp !(Maybe Alias)
|
||||
deriving (Show, Eq)
|
||||
|
||||
|
@ -46,32 +46,42 @@ data PGColValue
|
||||
| PGValUnknown !T.Text
|
||||
deriving (Show, Eq)
|
||||
|
||||
txtEncoder :: PGColValue -> S.SQLExp
|
||||
txtEncoder colVal = case colVal of
|
||||
PGValInteger i -> S.SELit $ T.pack $ show i
|
||||
PGValSmallInt i -> S.SELit $ T.pack $ show i
|
||||
PGValBigInt i -> S.SELit $ T.pack $ show i
|
||||
PGValFloat f -> S.SELit $ T.pack $ show f
|
||||
PGValDouble d -> S.SELit $ T.pack $ show d
|
||||
PGValNumeric sc -> S.SELit $ T.pack $ show sc
|
||||
PGValBoolean b -> S.SELit $ bool "false" "true" b
|
||||
PGValChar t -> S.SELit $ T.pack $ show t
|
||||
PGValVarchar t -> S.SELit t
|
||||
PGValText t -> S.SELit t
|
||||
PGValDate d -> S.SELit $ T.pack $ showGregorian d
|
||||
data TxtEncodedPGVal
|
||||
= TENull
|
||||
| TELit !Text
|
||||
deriving (Show, Eq)
|
||||
|
||||
txtEncodedPGVal :: PGColValue -> TxtEncodedPGVal
|
||||
txtEncodedPGVal colVal = case colVal of
|
||||
PGValInteger i -> TELit $ T.pack $ show i
|
||||
PGValSmallInt i -> TELit $ T.pack $ show i
|
||||
PGValBigInt i -> TELit $ T.pack $ show i
|
||||
PGValFloat f -> TELit $ T.pack $ show f
|
||||
PGValDouble d -> TELit $ T.pack $ show d
|
||||
PGValNumeric sc -> TELit $ T.pack $ show sc
|
||||
PGValBoolean b -> TELit $ bool "false" "true" b
|
||||
PGValChar t -> TELit $ T.pack $ show t
|
||||
PGValVarchar t -> TELit t
|
||||
PGValText t -> TELit t
|
||||
PGValDate d -> TELit $ T.pack $ showGregorian d
|
||||
PGValTimeStampTZ u ->
|
||||
S.SELit $ T.pack $ formatTime defaultTimeLocale "%FT%T%QZ" u
|
||||
TELit $ T.pack $ formatTime defaultTimeLocale "%FT%T%QZ" u
|
||||
PGValTimeTZ (ZonedTimeOfDay tod tz) ->
|
||||
S.SELit $ T.pack (show tod ++ timeZoneOffsetString tz)
|
||||
TELit $ T.pack (show tod ++ timeZoneOffsetString tz)
|
||||
PGNull _ ->
|
||||
S.SEUnsafe "NULL"
|
||||
PGValJSON (Q.JSON j) -> S.SELit $ TL.toStrict $
|
||||
TENull
|
||||
PGValJSON (Q.JSON j) -> TELit $ TL.toStrict $
|
||||
AE.encodeToLazyText j
|
||||
PGValJSONB (Q.JSONB j) -> S.SELit $ TL.toStrict $
|
||||
PGValJSONB (Q.JSONB j) -> TELit $ TL.toStrict $
|
||||
AE.encodeToLazyText j
|
||||
PGValGeo o -> S.SELit $ TL.toStrict $
|
||||
PGValGeo o -> TELit $ TL.toStrict $
|
||||
AE.encodeToLazyText o
|
||||
PGValUnknown t -> S.SELit t
|
||||
PGValUnknown t -> TELit t
|
||||
|
||||
txtEncoder :: PGColValue -> S.SQLExp
|
||||
txtEncoder colVal = case txtEncodedPGVal colVal of
|
||||
TENull -> S.SEUnsafe "NULL"
|
||||
TELit t -> S.SELit t
|
||||
|
||||
binEncoder :: PGColValue -> Q.PrepArg
|
||||
binEncoder colVal = case colVal of
|
||||
@ -110,7 +120,11 @@ binEncoder colVal = case colVal of
|
||||
PGValGeo o ->
|
||||
Q.toPrepVal $ TL.toStrict $ AE.encodeToLazyText o
|
||||
PGValUnknown t ->
|
||||
(PTI.auto, Just (TE.encodeUtf8 t, PQ.Text))
|
||||
textToPrepVal t
|
||||
|
||||
textToPrepVal :: Text -> Q.PrepArg
|
||||
textToPrepVal t =
|
||||
(PTI.auto, Just (TE.encodeUtf8 t, PQ.Text))
|
||||
|
||||
parsePGValue' :: PGColType
|
||||
-> Value
|
||||
@ -206,7 +220,7 @@ toPrepParam i ty =
|
||||
|
||||
toTxtValue :: PGColType -> PGColValue -> S.SQLExp
|
||||
toTxtValue ty val =
|
||||
S.annotateExp txtVal ty
|
||||
S.withTyAnn ty txtVal
|
||||
where
|
||||
txtVal = withGeoVal ty $ txtEncoder val
|
||||
|
||||
|
@ -31,6 +31,8 @@ import qualified Text.Mustache as M
|
||||
import qualified Text.Mustache.Compile as M
|
||||
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Hasura.GraphQL.Execute as E
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery as EL
|
||||
import qualified Hasura.GraphQL.Explain as GE
|
||||
import qualified Hasura.GraphQL.Schema as GS
|
||||
import qualified Hasura.GraphQL.Transport.HTTP as GH
|
||||
@ -92,8 +94,10 @@ mkConsoleHTML path authMode enableTelemetry =
|
||||
|
||||
data SchemaCacheRef
|
||||
= SchemaCacheRef
|
||||
{ _scrLock :: MVar ()
|
||||
, _scrCache :: IORef SchemaCache
|
||||
{ _scrLock :: MVar ()
|
||||
, _scrCache :: IORef (SchemaCache, SchemaCacheVer)
|
||||
-- an action to run when schemacache changes
|
||||
, _scrOnChange :: IO ()
|
||||
}
|
||||
|
||||
withSCUpdate
|
||||
@ -103,19 +107,20 @@ withSCUpdate scr action = do
|
||||
acquireLock
|
||||
(res, newSC) <- action `catchError` onError
|
||||
-- update schemacache in IO reference
|
||||
liftIO $ writeIORef cacheRef newSC
|
||||
liftIO $ modifyIORef' cacheRef $
|
||||
\(_, prevVer) -> (newSC, incSchemaCacheVer prevVer)
|
||||
liftIO onChange
|
||||
releaseLock
|
||||
return res
|
||||
where
|
||||
SchemaCacheRef lk cacheRef = scr
|
||||
SchemaCacheRef lk cacheRef onChange = scr
|
||||
onError e = releaseLock >> throwError e
|
||||
acquireLock = liftIO $ takeMVar lk
|
||||
releaseLock = liftIO $ putMVar lk ()
|
||||
|
||||
data ServerCtx
|
||||
= ServerCtx
|
||||
{ scIsolation :: Q.TxIsolation
|
||||
, scPGPool :: Q.PGPool
|
||||
{ scPGExecCtx :: PGExecCtx
|
||||
, scLogger :: L.Logger
|
||||
, scCacheRef :: SchemaCacheRef
|
||||
, scAuthMode :: AuthMode
|
||||
@ -123,6 +128,8 @@ data ServerCtx
|
||||
, scStringifyNum :: Bool
|
||||
, scEnabledAPIs :: S.HashSet API
|
||||
, scInstanceId :: InstanceId
|
||||
, scPlanCache :: E.PlanCache
|
||||
, scLQState :: EL.LiveQueriesState
|
||||
}
|
||||
|
||||
data HandlerCtx
|
||||
@ -159,7 +166,7 @@ buildQCtx :: Handler QCtx
|
||||
buildQCtx = do
|
||||
scRef <- scCacheRef . hcServerCtx <$> ask
|
||||
userInfo <- asks hcUser
|
||||
cache <- liftIO $ readIORef $ _scrCache scRef
|
||||
cache <- fmap fst $ liftIO $ readIORef $ _scrCache scRef
|
||||
strfyNum <- scStringifyNum . hcServerCtx <$> ask
|
||||
return $ QCtx userInfo cache $ SQLGenCtx strfyNum
|
||||
|
||||
@ -234,13 +241,12 @@ v1QueryHandler query = do
|
||||
dbAction = do
|
||||
userInfo <- asks hcUser
|
||||
scRef <- scCacheRef . hcServerCtx <$> ask
|
||||
schemaCache <- liftIO $ readIORef $ _scrCache scRef
|
||||
schemaCache <- fmap fst $ liftIO $ readIORef $ _scrCache scRef
|
||||
httpMgr <- scManager . hcServerCtx <$> ask
|
||||
strfyNum <- scStringifyNum . hcServerCtx <$> ask
|
||||
pool <- scPGPool . hcServerCtx <$> ask
|
||||
isoL <- scIsolation . hcServerCtx <$> ask
|
||||
pgExecCtx <- scPGExecCtx . hcServerCtx <$> ask
|
||||
instanceId <- scInstanceId . hcServerCtx <$> ask
|
||||
runQuery pool isoL instanceId userInfo schemaCache httpMgr strfyNum query
|
||||
runQuery pgExecCtx instanceId userInfo schemaCache httpMgr strfyNum query
|
||||
|
||||
-- Also update the schema cache
|
||||
dbActionReload = do
|
||||
@ -254,28 +260,28 @@ v1QueryHandler query = do
|
||||
newSc { scGCtxMap = mergedGCtxMap, scDefaultRemoteGCtx = defGCtx }
|
||||
return (resp, newSc')
|
||||
|
||||
v1Alpha1GQHandler :: GH.GraphQLRequest -> Handler EncJSON
|
||||
v1Alpha1GQHandler :: GH.GQLReqUnparsed -> Handler EncJSON
|
||||
v1Alpha1GQHandler query = do
|
||||
userInfo <- asks hcUser
|
||||
reqBody <- asks hcReqBody
|
||||
reqHeaders <- asks hcReqHeaders
|
||||
manager <- scManager . hcServerCtx <$> ask
|
||||
scRef <- scCacheRef . hcServerCtx <$> ask
|
||||
sc <- liftIO $ readIORef $ _scrCache scRef
|
||||
pool <- scPGPool . hcServerCtx <$> ask
|
||||
isoL <- scIsolation . hcServerCtx <$> ask
|
||||
(sc, scVer) <- liftIO $ readIORef $ _scrCache scRef
|
||||
pgExecCtx <- scPGExecCtx . hcServerCtx <$> ask
|
||||
strfyNum <- scStringifyNum . hcServerCtx <$> ask
|
||||
GH.runGQ pool isoL userInfo (SQLGenCtx strfyNum) sc manager reqHeaders query reqBody
|
||||
planCache <- scPlanCache . hcServerCtx <$> ask
|
||||
GH.runGQ pgExecCtx userInfo (SQLGenCtx strfyNum) planCache
|
||||
sc scVer manager reqHeaders query reqBody
|
||||
|
||||
gqlExplainHandler :: GE.GQLExplain -> Handler EncJSON
|
||||
gqlExplainHandler query = do
|
||||
onlyAdmin
|
||||
scRef <- scCacheRef . hcServerCtx <$> ask
|
||||
sc <- liftIO $ readIORef $ _scrCache scRef
|
||||
pool <- scPGPool . hcServerCtx <$> ask
|
||||
isoL <- scIsolation . hcServerCtx <$> ask
|
||||
sc <- fmap fst $ liftIO $ readIORef $ _scrCache scRef
|
||||
pgExecCtx <- scPGExecCtx . hcServerCtx <$> ask
|
||||
strfyNum <- scStringifyNum . hcServerCtx <$> ask
|
||||
GE.explainGQLQuery pool isoL sc (SQLGenCtx strfyNum) query
|
||||
GE.explainGQLQuery pgExecCtx sc (SQLGenCtx strfyNum) query
|
||||
|
||||
newtype QueryParser
|
||||
= QueryParser { getQueryParser :: QualifiedTable -> Handler RQLQuery }
|
||||
@ -317,34 +323,42 @@ mkWaiApp
|
||||
-> Q.PGPool -> HTTP.Manager -> AuthMode
|
||||
-> CorsConfig -> Bool -> Bool
|
||||
-> InstanceId -> S.HashSet API
|
||||
-> EL.LQOpts
|
||||
-> IO (Wai.Application, SchemaCacheRef, Maybe UTCTime)
|
||||
mkWaiApp isoLevel loggerCtx strfyNum pool httpManager mode corsCfg
|
||||
enableConsole enableTelemetry instanceId apis = do
|
||||
enableConsole enableTelemetry instanceId apis
|
||||
lqOpts = do
|
||||
let pgExecCtx = PGExecCtx pool isoLevel
|
||||
pgExecCtxSer = PGExecCtx pool Q.Serializable
|
||||
(cacheRef, cacheBuiltTime) <- do
|
||||
pgResp <- runExceptT $ peelRun emptySchemaCache adminUserInfo
|
||||
httpManager strfyNum pool Q.Serializable $ do
|
||||
httpManager strfyNum pgExecCtxSer $ do
|
||||
buildSchemaCache
|
||||
liftTx fetchLastUpdate
|
||||
(time, sc) <- either initErrExit return pgResp
|
||||
scRef <- newIORef sc
|
||||
scRef <- newIORef (sc, initSchemaCacheVer)
|
||||
return (scRef, snd <$> time)
|
||||
|
||||
cacheLock <- newMVar ()
|
||||
planCache <- E.initPlanCache
|
||||
|
||||
let schemaCacheRef = SchemaCacheRef cacheLock cacheRef
|
||||
serverCtx = ServerCtx isoLevel pool (L.mkLogger loggerCtx)
|
||||
schemaCacheRef mode httpManager strfyNum apis instanceId
|
||||
let corsPolicy = mkDefaultCorsPolicy corsCfg
|
||||
sqlGenCtx = SQLGenCtx strfyNum
|
||||
logger = L.mkLogger loggerCtx
|
||||
|
||||
lqState <- EL.initLiveQueriesState lqOpts pgExecCtx
|
||||
wsServerEnv <- WS.createWSServerEnv logger pgExecCtx lqState
|
||||
cacheRef httpManager corsPolicy sqlGenCtx planCache
|
||||
|
||||
let schemaCacheRef =
|
||||
SchemaCacheRef cacheLock cacheRef (E.clearPlanCache planCache)
|
||||
serverCtx = ServerCtx pgExecCtx logger
|
||||
schemaCacheRef mode httpManager
|
||||
strfyNum apis instanceId planCache lqState
|
||||
|
||||
spockApp <- spockAsApp $ spockT id $
|
||||
httpApp corsCfg serverCtx enableConsole enableTelemetry
|
||||
|
||||
let runTx tx = runExceptT $ runLazyTx pool isoLevel tx
|
||||
corsPolicy = mkDefaultCorsPolicy corsCfg
|
||||
sqlGenCtx = SQLGenCtx strfyNum
|
||||
|
||||
wsServerEnv <- WS.createWSServerEnv (scLogger serverCtx) httpManager sqlGenCtx
|
||||
cacheRef runTx corsPolicy
|
||||
|
||||
let wsServerApp = WS.createWSServerApp mode wsServerEnv
|
||||
return ( WS.websocketsOr WS.defaultConnectionOptions wsServerApp spockApp
|
||||
, schemaCacheRef
|
||||
@ -387,8 +401,14 @@ httpApp corsCfg serverCtx enableConsole enableTelemetry = do
|
||||
query <- parseBody
|
||||
v1Alpha1GQHandler query
|
||||
|
||||
-- get "v1alpha1/graphql/schema" $
|
||||
-- mkSpockAction encodeQErr serverCtx v1Alpha1GQSchemaHandler
|
||||
#ifdef InternalAPIs
|
||||
get "internal/plan_cache" $ do
|
||||
respJ <- liftIO $ E.dumpPlanCache $ scPlanCache serverCtx
|
||||
json respJ
|
||||
get "internal/subscriptions" $ do
|
||||
respJ <- liftIO $ EL.dumpLiveQueriesState $ scLQState serverCtx
|
||||
json respJ
|
||||
#endif
|
||||
|
||||
forM_ [GET,POST] $ \m -> hookAny m $ \_ -> do
|
||||
let qErr = err404 NotFound "resource does not exist"
|
||||
|
@ -1,20 +1,21 @@
|
||||
module Hasura.Server.Init where
|
||||
|
||||
import qualified Database.PG.Query as Q
|
||||
import qualified Database.PG.Query as Q
|
||||
|
||||
import Options.Applicative
|
||||
|
||||
import qualified Data.Aeson as J
|
||||
import qualified Data.HashSet as Set
|
||||
import qualified Data.String as DataString
|
||||
import qualified Data.Text as T
|
||||
import qualified Data.UUID as UUID
|
||||
import qualified Data.UUID.V4 as UUID
|
||||
import qualified Hasura.Logging as L
|
||||
import qualified Text.PrettyPrint.ANSI.Leijen as PP
|
||||
import qualified Data.Aeson as J
|
||||
import qualified Data.HashSet as Set
|
||||
import qualified Data.String as DataString
|
||||
import qualified Data.Text as T
|
||||
import qualified Data.UUID as UUID
|
||||
import qualified Data.UUID.V4 as UUID
|
||||
import qualified Hasura.GraphQL.Execute.LiveQuery as LQ
|
||||
import qualified Hasura.Logging as L
|
||||
import qualified Text.PrettyPrint.ANSI.Leijen as PP
|
||||
|
||||
import Hasura.Prelude
|
||||
import Hasura.RQL.Types (RoleName (..))
|
||||
import Hasura.RQL.Types (RoleName (..))
|
||||
import Hasura.Server.Auth
|
||||
import Hasura.Server.Cors
|
||||
import Hasura.Server.Logging
|
||||
@ -40,20 +41,23 @@ type RawAuthHook = AuthHookG (Maybe T.Text) (Maybe AuthHookType)
|
||||
|
||||
data RawServeOptions
|
||||
= RawServeOptions
|
||||
{ rsoPort :: !(Maybe Int)
|
||||
, rsoHost :: !(Maybe HostPreference)
|
||||
, rsoConnParams :: !RawConnParams
|
||||
, rsoTxIso :: !(Maybe Q.TxIsolation)
|
||||
, rsoAdminSecret :: !(Maybe AdminSecret)
|
||||
, rsoAuthHook :: !RawAuthHook
|
||||
, rsoJwtSecret :: !(Maybe Text)
|
||||
, rsoUnAuthRole :: !(Maybe RoleName)
|
||||
, rsoCorsConfig :: !(Maybe CorsConfig)
|
||||
, rsoEnableConsole :: !Bool
|
||||
, rsoEnableTelemetry :: !(Maybe Bool)
|
||||
, rsoWsReadCookie :: !Bool
|
||||
, rsoStringifyNum :: !Bool
|
||||
, rsoEnabledAPIs :: !(Maybe [API])
|
||||
{ rsoPort :: !(Maybe Int)
|
||||
, rsoHost :: !(Maybe HostPreference)
|
||||
, rsoConnParams :: !RawConnParams
|
||||
, rsoTxIso :: !(Maybe Q.TxIsolation)
|
||||
, rsoAdminSecret :: !(Maybe AdminSecret)
|
||||
, rsoAuthHook :: !RawAuthHook
|
||||
, rsoJwtSecret :: !(Maybe Text)
|
||||
, rsoUnAuthRole :: !(Maybe RoleName)
|
||||
, rsoCorsConfig :: !(Maybe CorsConfig)
|
||||
, rsoEnableConsole :: !Bool
|
||||
, rsoEnableTelemetry :: !(Maybe Bool)
|
||||
, rsoWsReadCookie :: !Bool
|
||||
, rsoStringifyNum :: !Bool
|
||||
, rsoEnabledAPIs :: !(Maybe [API])
|
||||
, rsoMxRefetchInt :: !(Maybe LQ.RefetchInterval)
|
||||
, rsoMxBatchSize :: !(Maybe LQ.BatchSize)
|
||||
, rsoFallbackRefetchInt :: !(Maybe LQ.RefetchInterval)
|
||||
} deriving (Show, Eq)
|
||||
|
||||
data ServeOptions
|
||||
@ -71,6 +75,7 @@ data ServeOptions
|
||||
, soEnableTelemetry :: !Bool
|
||||
, soStringifyNum :: !Bool
|
||||
, soEnabledAPIs :: !(Set.HashSet API)
|
||||
, soLiveQueryOpts :: !LQ.LQOpts
|
||||
} deriving (Show, Eq)
|
||||
|
||||
data RawConnInfo =
|
||||
@ -150,6 +155,12 @@ instance FromEnv CorsConfig where
|
||||
instance FromEnv [API] where
|
||||
fromEnv = readAPIs
|
||||
|
||||
instance FromEnv LQ.BatchSize where
|
||||
fromEnv = fmap LQ.mkBatchSize . readEither
|
||||
|
||||
instance FromEnv LQ.RefetchInterval where
|
||||
fromEnv = fmap LQ.refetchIntervalFromMilli . readEither
|
||||
|
||||
parseStrAsBool :: String -> Either String Bool
|
||||
parseStrAsBool t
|
||||
| t `elem` truthVals = Right True
|
||||
@ -253,8 +264,10 @@ mkServeOptions rso = do
|
||||
strfyNum <- withEnvBool (rsoStringifyNum rso) $ fst stringifyNumEnv
|
||||
enabledAPIs <- Set.fromList . fromMaybe [METADATA,GRAPHQL] <$>
|
||||
withEnv (rsoEnabledAPIs rso) (fst enabledAPIsEnv)
|
||||
lqOpts <- mkLQOpts
|
||||
return $ ServeOptions port host connParams txIso adminScrt authHook jwtSecret
|
||||
unAuthRole corsCfg enableConsole enableTelemetry strfyNum enabledAPIs
|
||||
unAuthRole corsCfg enableConsole
|
||||
enableTelemetry strfyNum enabledAPIs lqOpts
|
||||
where
|
||||
mkConnParams (RawConnParams s c i p) = do
|
||||
stripes <- fromMaybe 1 <$> withEnv s (fst pgStripesEnv)
|
||||
@ -286,6 +299,16 @@ mkServeOptions rso = do
|
||||
CCDisabled _ -> CCDisabled wsReadCookie
|
||||
_ -> corsCfg
|
||||
|
||||
mkLQOpts = do
|
||||
mxRefetchIntM <- withEnv (rsoMxRefetchInt rso) $
|
||||
fst mxRefetchDelayEnv
|
||||
mxBatchSizeM <- withEnv (rsoMxBatchSize rso) $
|
||||
fst mxBatchSizeEnv
|
||||
fallbackRefetchIntM <- withEnv (rsoFallbackRefetchInt rso) $
|
||||
fst fallbackRefetchDelayEnv
|
||||
return $ LQ.mkLQOpts (LQ.mkMxOpts mxBatchSizeM mxRefetchIntM)
|
||||
(LQ.mkFallbackOpts fallbackRefetchIntM)
|
||||
|
||||
|
||||
mkExamplesDoc :: [[String]] -> PP.Doc
|
||||
mkExamplesDoc exampleLines =
|
||||
@ -751,6 +774,54 @@ parseEnabledAPIs = optional $
|
||||
help (snd enabledAPIsEnv)
|
||||
)
|
||||
|
||||
parseMxRefetchInt :: Parser (Maybe LQ.RefetchInterval)
|
||||
parseMxRefetchInt =
|
||||
optional $
|
||||
option (eitherReader fromEnv)
|
||||
( long "live-queries-multiplexed-refetch-interval" <>
|
||||
metavar "<INTERVAL(ms)>" <>
|
||||
help (snd mxRefetchDelayEnv)
|
||||
)
|
||||
|
||||
parseMxBatchSize :: Parser (Maybe LQ.BatchSize)
|
||||
parseMxBatchSize =
|
||||
optional $
|
||||
option (eitherReader fromEnv)
|
||||
( long "live-queries-multiplexed-batch-size" <>
|
||||
metavar "BATCH_SIZE" <>
|
||||
help (snd mxBatchSizeEnv)
|
||||
)
|
||||
|
||||
mxRefetchDelayEnv :: (String, String)
|
||||
mxRefetchDelayEnv =
|
||||
( "HASURA_GRAPHQL_LIVE_QUERIES_MULTIPLEXED_REFETCH_INTERVAL"
|
||||
, "results will only be sent once in this interval (in milliseconds) for \
|
||||
\live queries which can be multiplexed. Default: 1000 (1sec)"
|
||||
)
|
||||
|
||||
mxBatchSizeEnv :: (String, String)
|
||||
mxBatchSizeEnv =
|
||||
( "HASURA_GRAPHQL_LIVE_QUERIES_MULTIPLEXED_BATCH_SIZE"
|
||||
, "multiplexed live queries are split into batches of the specified \
|
||||
\size. Default 100. "
|
||||
)
|
||||
|
||||
parseFallbackRefetchInt :: Parser (Maybe LQ.RefetchInterval)
|
||||
parseFallbackRefetchInt =
|
||||
optional $
|
||||
option (eitherReader fromEnv)
|
||||
( long "live-queries-fallback-refetch-interval" <>
|
||||
metavar "<INTERVAL(ms)>" <>
|
||||
help (snd mxRefetchDelayEnv)
|
||||
)
|
||||
|
||||
fallbackRefetchDelayEnv :: (String, String)
|
||||
fallbackRefetchDelayEnv =
|
||||
( "HASURA_GRAPHQL_LIVE_QUERIES_FALLBACK_REFETCH_INTERVAL"
|
||||
, "results will only be sent once in this interval (in milliseconds) for \
|
||||
\live queries which cannot be multiplexed. Default: 1000 (1sec)"
|
||||
)
|
||||
|
||||
-- Init logging related
|
||||
connInfoToLog :: Q.ConnInfo -> StartupLog
|
||||
connInfoToLog (Q.ConnInfo host port user _ db _ retries) =
|
||||
|
@ -142,28 +142,28 @@ peelRun
|
||||
-> UserInfo
|
||||
-> HTTP.Manager
|
||||
-> Bool
|
||||
-> Q.PGPool -> Q.TxIsolation
|
||||
-> PGExecCtx
|
||||
-> Run a -> ExceptT QErr IO (a, SchemaCache)
|
||||
peelRun sc userInfo httMgr strfyNum pgPool txIso (Run m) =
|
||||
runLazyTx pgPool txIso $ withUserInfo userInfo lazyTx
|
||||
peelRun sc userInfo httMgr strfyNum pgExecCtx (Run m) =
|
||||
runLazyTx pgExecCtx $ withUserInfo userInfo lazyTx
|
||||
where
|
||||
sqlGenCtx = SQLGenCtx strfyNum
|
||||
lazyTx = runReaderT (runStateT m sc) (userInfo, httMgr, sqlGenCtx)
|
||||
|
||||
runQuery
|
||||
:: (MonadIO m, MonadError QErr m)
|
||||
=> Q.PGPool -> Q.TxIsolation -> InstanceId
|
||||
=> PGExecCtx -> InstanceId
|
||||
-> UserInfo -> SchemaCache -> HTTP.Manager
|
||||
-> Bool -> RQLQuery -> m (EncJSON, SchemaCache)
|
||||
runQuery pool isoL instanceId userInfo sc hMgr strfyNum query = do
|
||||
runQuery pgExecCtx instanceId userInfo sc hMgr strfyNum query = do
|
||||
resE <- liftIO $ runExceptT $
|
||||
peelRun sc userInfo hMgr strfyNum pool isoL $ runQueryM query
|
||||
peelRun sc userInfo hMgr strfyNum pgExecCtx $ runQueryM query
|
||||
either throwError withReload resE
|
||||
where
|
||||
withReload r = do
|
||||
when (queryNeedsReload query) $ do
|
||||
e <- liftIO $ runExceptT $ Q.runTx pool (isoL, Nothing)
|
||||
$ recordSchemaUpdate instanceId
|
||||
e <- liftIO $ runExceptT $ runLazyTx pgExecCtx
|
||||
$ liftTx $ recordSchemaUpdate instanceId
|
||||
liftEither e
|
||||
return r
|
||||
|
||||
|
@ -194,7 +194,7 @@ refreshSchemaCache strfyNum pool logger httpManager cacheRef threadType msg = do
|
||||
-- Reload schema cache from catalog
|
||||
resE <- liftIO $ runExceptT $ withSCUpdate cacheRef $
|
||||
peelRun emptySchemaCache adminUserInfo
|
||||
httpManager strfyNum pool PG.Serializable buildSchemaCache
|
||||
httpManager strfyNum (PGExecCtx pool PG.Serializable) buildSchemaCache
|
||||
case resE of
|
||||
Left e -> logError logger threadType $ TEQueryError e
|
||||
Right _ ->
|
||||
|
@ -94,13 +94,13 @@ mkPayload dbId instanceId version metrics =
|
||||
runTelemetry
|
||||
:: Logger
|
||||
-> HTTP.Manager
|
||||
-> IORef SchemaCache
|
||||
-> IORef (SchemaCache, SchemaCacheVer)
|
||||
-> (Text, Text)
|
||||
-> IO ()
|
||||
runTelemetry (Logger logger) manager cacheRef (dbId, instanceId) = do
|
||||
let options = wreqOptions manager []
|
||||
forever $ do
|
||||
schemaCache <- readIORef cacheRef
|
||||
schemaCache <- fmap fst $ readIORef cacheRef
|
||||
let metrics = computeMetrics schemaCache
|
||||
payload = A.encode $ mkPayload dbId instanceId currentVersion metrics
|
||||
logger $ debugLBS $ "metrics_info: " <> payload
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
# Specifies the GHC version and set of packages available (e.g., lts-3.5, nightly-2015-09-21, ghc-7.10.2)
|
||||
# resolver: lts-10.8
|
||||
resolver: lts-13.12
|
||||
resolver: lts-13.16
|
||||
# Local packages, usually specified by relative directory name
|
||||
packages:
|
||||
- '.'
|
||||
@ -13,7 +13,7 @@ packages:
|
||||
# Packages to be pulled from upstream that are not in the resolver (e.g., acme-missiles-0.3)
|
||||
extra-deps:
|
||||
- git: https://github.com/hasura/pg-client-hs.git
|
||||
commit: ccc345b634e92dcbda53dfc84d86fc9381e0c871
|
||||
commit: 85f9c2c15e4fa09f2e2a86dbb23149b5256bdd34
|
||||
- git: https://github.com/hasura/graphql-parser-hs.git
|
||||
commit: ff95d9a96aa5ef9e5390f8712958e4118e3831f6
|
||||
- ginger-0.8.4.0
|
||||
|
@ -218,7 +218,7 @@ class TestAddRemoteSchemaTbls:
|
||||
q = mk_add_remote_q('simple2-graphql', 'http://localhost:5000/country-graphql')
|
||||
st_code, resp = hge_ctx.v1q(q)
|
||||
assert st_code == 500, resp
|
||||
assert resp['code'] == 'postgres-error'
|
||||
assert resp['code'] == 'unexpected'
|
||||
|
||||
def test_add_schema_same_type_containing_same_scalar(self, hge_ctx):
|
||||
"""
|
||||
|
Loading…
Reference in New Issue
Block a user