module Hasura.GraphQL.Execute.RemoteJoin.Join
  ( processRemoteJoins,
    foldJoinTreeWith,
  )
where

import Control.Lens (view, _3)
import Control.Monad.Trans.Control
import Data.Aeson.Ordered qualified as JO
import Data.ByteString.Lazy qualified as BL
import Data.Environment qualified as Env
import Data.HashMap.Strict.Extended qualified as HashMap
import Data.HashMap.Strict.InsOrd qualified as InsOrdHashMap
import Data.HashMap.Strict.NonEmpty qualified as NEMap
import Data.HashSet qualified as HS
import Data.IntMap.Strict qualified as IntMap
import Data.Text qualified as T
import Data.Tuple (swap)
import Hasura.Backends.DataConnector.Agent.Client (AgentLicenseKey)
import Hasura.Base.Error
import Hasura.CredentialCache
import Hasura.EncJSON
import Hasura.GraphQL.Execute.Backend qualified as EB
import Hasura.GraphQL.Execute.Instances ()
import Hasura.GraphQL.Execute.RemoteJoin.RemoteSchema qualified as RS
import Hasura.GraphQL.Execute.RemoteJoin.Source qualified as S
import Hasura.GraphQL.Execute.RemoteJoin.Types
import Hasura.GraphQL.Logging (MonadExecutionLog, MonadQueryLog, statsToAnyBackend)
import Hasura.GraphQL.RemoteServer (execRemoteGQ)
import Hasura.GraphQL.Transport.Backend qualified as TB
import Hasura.GraphQL.Transport.HTTP.Protocol (GQLReqOutgoing, GQLReqUnparsed, _grOperationName, _unOperationName)
import Hasura.GraphQL.Transport.Instances ()
import Hasura.Logging qualified as L
import Hasura.Prelude
import Hasura.QueryTags
import Hasura.RQL.Types.Common
import Hasura.RemoteSchema.SchemaCache
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.Server.Types (RequestId)
import Hasura.Services.Network
import Hasura.Session
import Hasura.Tracing qualified as Tracing
import Language.GraphQL.Draft.Syntax qualified as G
import Network.HTTP.Types qualified as HTTP

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

-- | Process all remote joins, recursively.
--
-- Given the result of the first step of an execution and its associated remote
-- joins, process all joins recursively to build the resulting JSON object.
--
-- This function is a thin wrapper around 'processRemoteJoinsWith', and starts
-- the join tree traversal process by re-parsing the 'EncJSON' value into an
-- introspectable JSON 'Value', and "injects" the required functions to process
-- each join over the network.
processRemoteJoins ::
  forall m.
  ( MonadError QErr m,
    MonadIO m,
    MonadBaseControl IO m,
    MonadQueryTags m,
    MonadQueryLog m,
    MonadExecutionLog m,
    Tracing.MonadTrace m,
    ProvidesNetwork m
  ) =>
  RequestId ->
  L.Logger L.Hasura ->
  Maybe (CredentialCache AgentLicenseKey) ->
  Env.Environment ->
  [HTTP.Header] ->
  UserInfo ->
  EncJSON ->
  Maybe RemoteJoins ->
  GQLReqUnparsed ->
  m EncJSON
processRemoteJoins :: forall (m :: * -> *).
(MonadError QErr m, MonadIO m, MonadBaseControl IO m,
 MonadQueryTags m, MonadQueryLog m, MonadExecutionLog m,
 MonadTrace m, ProvidesNetwork m) =>
RequestId
-> Logger Hasura
-> Maybe (CredentialCache AgentLicenseKey)
-> Environment
-> [Header]
-> UserInfo
-> EncJSON
-> Maybe RemoteJoins
-> GQLReqUnparsed
-> m EncJSON
processRemoteJoins RequestId
requestId Logger Hasura
logger Maybe (CredentialCache AgentLicenseKey)
agentLicenseKey Environment
env [Header]
requestHeaders UserInfo
userInfo EncJSON
lhs Maybe RemoteJoins
maybeJoinTree GQLReqUnparsed
gqlreq =
  Text -> m EncJSON -> m EncJSON
forall (m :: * -> *) a.
(MonadIO m, MonadTrace m) =>
Text -> m a -> m a
Tracing.newSpan Text
"Process remote joins" (m EncJSON -> m EncJSON) -> m EncJSON -> m EncJSON
forall a b. (a -> b) -> a -> b
$ Maybe RemoteJoins
-> EncJSON -> (RemoteJoins -> m EncJSON) -> m EncJSON
forall (f :: * -> *) a.
Applicative f =>
Maybe RemoteJoins -> a -> (RemoteJoins -> f a) -> f a
forRemoteJoins Maybe RemoteJoins
maybeJoinTree EncJSON
lhs \RemoteJoins
joinTree -> do
    Value
lhsParsed <-
      ByteString -> Either String Value
JO.eitherDecode (EncJSON -> ByteString
encJToLBS EncJSON
lhs)
        Either String Value -> (String -> m Value) -> m Value
forall (m :: * -> *) e a.
Applicative m =>
Either e a -> (e -> m a) -> m a
`onLeft` (Text -> m Value
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 (Text -> m Value) -> (String -> Text) -> String -> m Value
forall b c a. (b -> c) -> (a -> b) -> a -> c
. String -> Text
T.pack)
    Identity Value
jsonResult <-
      (AnyBackend SourceJoinCall -> m ByteString)
-> (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString)
-> UserInfo
-> Identity Value
-> RemoteJoins
-> [Header]
-> Maybe Name
-> m (Identity Value)
forall (m :: * -> *) (f :: * -> *).
(MonadError QErr m, MonadQueryTags m, Traversable f, MonadTrace m,
 MonadIO m) =>
(AnyBackend SourceJoinCall -> m ByteString)
-> (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString)
-> UserInfo
-> f Value
-> RemoteJoins
-> [Header]
-> Maybe Name
-> m (f Value)
foldJoinTreeWith
        AnyBackend SourceJoinCall -> m ByteString
callSource
        ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString
callRemoteServer
        UserInfo
userInfo
        (Value -> Identity Value
forall a. a -> Identity a
Identity Value
lhsParsed)
        RemoteJoins
joinTree
        [Header]
requestHeaders
        (OperationName -> Name
_unOperationName (OperationName -> Name) -> Maybe OperationName -> Maybe Name
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> GQLReqUnparsed -> Maybe OperationName
forall a. GQLReq a -> Maybe OperationName
_grOperationName GQLReqUnparsed
gqlreq)
    EncJSON -> m EncJSON
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (EncJSON -> m EncJSON) -> EncJSON -> m EncJSON
forall a b. (a -> b) -> a -> b
$ Value -> EncJSON
encJFromOrderedValue (Value -> EncJSON) -> Value -> EncJSON
forall a b. (a -> b) -> a -> b
$ Identity Value -> Value
forall a. Identity a -> a
runIdentity Identity Value
jsonResult
  where
    -- How to process a source join call over the network.
    callSource ::
      -- Generated information about the step
      AB.AnyBackend S.SourceJoinCall ->
      -- Resulting JSON object, as a 'ByteString'.
      m BL.ByteString
    callSource :: AnyBackend SourceJoinCall -> m ByteString
callSource AnyBackend SourceJoinCall
sourceJoinCall =
      forall (c :: BackendType -> Constraint) (i :: BackendType -> *) r.
AllBackendsSatisfy c =>
AnyBackend i -> (forall (b :: BackendType). c b => i b -> r) -> r
AB.dispatchAnyBackend @TB.BackendTransport AnyBackend SourceJoinCall
sourceJoinCall \(S.SourceJoinCall {SourceConfig b
RootFieldAlias
DBStepInfo b
_sjcRootFieldAlias :: RootFieldAlias
_sjcSourceConfig :: SourceConfig b
_sjcStepInfo :: DBStepInfo b
_sjcRootFieldAlias :: forall (b :: BackendType). SourceJoinCall b -> RootFieldAlias
_sjcSourceConfig :: forall (b :: BackendType). SourceJoinCall b -> SourceConfig b
_sjcStepInfo :: forall (b :: BackendType). SourceJoinCall b -> DBStepInfo b
..} :: S.SourceJoinCall b) -> do
        (DiffTime, EncJSON)
response <-
          forall (b :: BackendType) (m :: * -> *).
(BackendTransport b, MonadIO m, MonadBaseControl IO m,
 MonadError QErr m, MonadQueryLog m, MonadExecutionLog m,
 MonadTrace m) =>
RequestId
-> GQLReqUnparsed
-> RootFieldAlias
-> UserInfo
-> Logger Hasura
-> Maybe (CredentialCache AgentLicenseKey)
-> SourceConfig b
-> OnBaseMonad
     (ExecutionMonad b) (Maybe (AnyBackend ExecutionStats), EncJSON)
-> Maybe (PreparedQuery b)
-> ResolvedConnectionTemplate b
-> m (DiffTime, EncJSON)
TB.runDBQuery @b
            RequestId
requestId
            GQLReqUnparsed
gqlreq
            RootFieldAlias
_sjcRootFieldAlias
            UserInfo
userInfo
            Logger Hasura
logger
            Maybe (CredentialCache AgentLicenseKey)
agentLicenseKey
            SourceConfig b
_sjcSourceConfig
            ((ActionResult b -> (Maybe (AnyBackend ExecutionStats), EncJSON))
-> OnBaseMonad (ExecutionMonad b) (ActionResult b)
-> OnBaseMonad
     (ExecutionMonad b) (Maybe (AnyBackend ExecutionStats), EncJSON)
forall a b.
(a -> b)
-> OnBaseMonad (ExecutionMonad b) a
-> OnBaseMonad (ExecutionMonad b) b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (forall (b :: BackendType).
HasTag b =>
ActionResult b -> (Maybe (AnyBackend ExecutionStats), EncJSON)
statsToAnyBackend @b) (DBStepInfo b -> OnBaseMonad (ExecutionMonad b) (ActionResult b)
forall (b :: BackendType).
DBStepInfo b -> OnBaseMonad (ExecutionMonad b) (ActionResult b)
EB.dbsiAction DBStepInfo b
_sjcStepInfo))
            (DBStepInfo b -> Maybe (PreparedQuery b)
forall (b :: BackendType). DBStepInfo b -> Maybe (PreparedQuery b)
EB.dbsiPreparedQuery DBStepInfo b
_sjcStepInfo)
            (DBStepInfo b -> ResolvedConnectionTemplate b
forall (b :: BackendType).
DBStepInfo b -> ResolvedConnectionTemplate b
EB.dbsiResolvedConnectionTemplate DBStepInfo b
_sjcStepInfo)
        ByteString -> m ByteString
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ByteString -> m ByteString) -> ByteString -> m ByteString
forall a b. (a -> b) -> a -> b
$ EncJSON -> ByteString
encJToLBS (EncJSON -> ByteString) -> EncJSON -> ByteString
forall a b. (a -> b) -> a -> b
$ (DiffTime, EncJSON) -> EncJSON
forall a b. (a, b) -> b
snd (DiffTime, EncJSON)
response

    -- How to process a remote schema join call over the network.
    callRemoteServer ::
      -- Information about the remote schema
      ValidatedRemoteSchemaDef ->
      -- Generated GraphQL request
      GQLReqOutgoing ->
      -- Resulting JSON object, as a 'ByteString'.
      m BL.ByteString
    callRemoteServer :: ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString
callRemoteServer ValidatedRemoteSchemaDef
remoteSchemaInfo GQLReqOutgoing
request =
      ((DiffTime, [Header], ByteString) -> ByteString)
-> m (DiffTime, [Header], ByteString) -> m ByteString
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Getting ByteString (DiffTime, [Header], ByteString) ByteString
-> (DiffTime, [Header], ByteString) -> ByteString
forall s (m :: * -> *) a. MonadReader s m => Getting a s a -> m a
view Getting ByteString (DiffTime, [Header], ByteString) ByteString
forall s t a b. Field3 s t a b => Lens s t a b
Lens
  (DiffTime, [Header], ByteString)
  (DiffTime, [Header], ByteString)
  ByteString
  ByteString
_3)
        (m (DiffTime, [Header], ByteString) -> m ByteString)
-> m (DiffTime, [Header], ByteString) -> m ByteString
forall a b. (a -> b) -> a -> b
$ Environment
-> UserInfo
-> [Header]
-> ValidatedRemoteSchemaDef
-> GQLReqOutgoing
-> m (DiffTime, [Header], ByteString)
forall (m :: * -> *).
(MonadIO m, MonadError QErr m, MonadTrace m, ProvidesNetwork m) =>
Environment
-> UserInfo
-> [Header]
-> ValidatedRemoteSchemaDef
-> GQLReqOutgoing
-> m (DiffTime, [Header], ByteString)
execRemoteGQ Environment
env UserInfo
userInfo [Header]
requestHeaders ValidatedRemoteSchemaDef
remoteSchemaInfo GQLReqOutgoing
request

-- | Fold the join tree.
--
-- This function takes as an argument the functions that will be used to do the
-- actual network calls; this allows this function not to require 'MonadIO',
-- allowing it to be used in tests.
foldJoinTreeWith ::
  ( MonadError QErr m,
    MonadQueryTags m,
    Traversable f,
    Tracing.MonadTrace m,
    MonadIO m
  ) =>
  -- | How to process a call to a source.
  (AB.AnyBackend S.SourceJoinCall -> m BL.ByteString) ->
  -- | How to process a call to a remote schema.
  (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m BL.ByteString) ->
  -- | User information.
  UserInfo ->
  -- | Initial accumulator; the LHS of this join tree.
  (f JO.Value) ->
  RemoteJoins ->
  [HTTP.Header] ->
  Maybe G.Name ->
  m (f JO.Value)
foldJoinTreeWith :: forall (m :: * -> *) (f :: * -> *).
(MonadError QErr m, MonadQueryTags m, Traversable f, MonadTrace m,
 MonadIO m) =>
(AnyBackend SourceJoinCall -> m ByteString)
-> (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString)
-> UserInfo
-> f Value
-> RemoteJoins
-> [Header]
-> Maybe Name
-> m (f Value)
foldJoinTreeWith AnyBackend SourceJoinCall -> m ByteString
callSource ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString
callRemoteSchema UserInfo
userInfo f Value
lhs RemoteJoins
joinTree [Header]
reqHeaders Maybe Name
operationName = do
  (f (CompositeValue ReplacementToken)
compositeValue, IntMap JoinArguments
joins) <- JoinTree (JoinCallId, RemoteJoin)
-> f Value
-> m (f (CompositeValue ReplacementToken), IntMap JoinArguments)
forall (f :: * -> *) (m :: * -> *).
(MonadError QErr m, Traversable f) =>
JoinTree (JoinCallId, RemoteJoin)
-> f Value
-> m (f (CompositeValue ReplacementToken), IntMap JoinArguments)
collectJoinArguments (RemoteJoins -> JoinTree (JoinCallId, RemoteJoin)
assignJoinIds RemoteJoins
joinTree) f Value
lhs
  IntMap (IntMap Value)
joinIndices <- (IntMap (Maybe (IntMap Value)) -> IntMap (IntMap Value))
-> m (IntMap (Maybe (IntMap Value))) -> m (IntMap (IntMap Value))
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap IntMap (Maybe (IntMap Value)) -> IntMap (IntMap Value)
forall a. IntMap (Maybe a) -> IntMap a
forall (f :: * -> *) a. Filterable f => f (Maybe a) -> f a
catMaybes
    (m (IntMap (Maybe (IntMap Value))) -> m (IntMap (IntMap Value)))
-> m (IntMap (Maybe (IntMap Value))) -> m (IntMap (IntMap Value))
forall a b. (a -> b) -> a -> b
$ IntMap JoinArguments
-> (JoinArguments -> m (Maybe (IntMap Value)))
-> m (IntMap (Maybe (IntMap Value)))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
t a -> (a -> f b) -> f (t b)
for IntMap JoinArguments
joins
    ((JoinArguments -> m (Maybe (IntMap Value)))
 -> m (IntMap (Maybe (IntMap Value))))
-> (JoinArguments -> m (Maybe (IntMap Value)))
-> m (IntMap (Maybe (IntMap Value)))
forall a b. (a -> b) -> a -> b
$ \JoinArguments {HashMap JoinArgument JoinCallId
FieldName
RemoteJoin
_jalJoin :: RemoteJoin
_jalArguments :: HashMap JoinArgument JoinCallId
_jalFieldName :: FieldName
_jalJoin :: JoinArguments -> RemoteJoin
_jalArguments :: JoinArguments -> HashMap JoinArgument JoinCallId
_jalFieldName :: JoinArguments -> FieldName
..} -> do
      let joinArguments :: IntMap JoinArgument
joinArguments = [(JoinCallId, JoinArgument)] -> IntMap JoinArgument
forall a. [(JoinCallId, a)] -> IntMap a
IntMap.fromList ([(JoinCallId, JoinArgument)] -> IntMap JoinArgument)
-> [(JoinCallId, JoinArgument)] -> IntMap JoinArgument
forall a b. (a -> b) -> a -> b
$ ((JoinArgument, JoinCallId) -> (JoinCallId, JoinArgument))
-> [(JoinArgument, JoinCallId)] -> [(JoinCallId, JoinArgument)]
forall a b. (a -> b) -> [a] -> [b]
map (JoinArgument, JoinCallId) -> (JoinCallId, JoinArgument)
forall a b. (a, b) -> (b, a)
swap ([(JoinArgument, JoinCallId)] -> [(JoinCallId, JoinArgument)])
-> [(JoinArgument, JoinCallId)] -> [(JoinCallId, JoinArgument)]
forall a b. (a -> b) -> a -> b
$ HashMap JoinArgument JoinCallId -> [(JoinArgument, JoinCallId)]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList HashMap JoinArgument JoinCallId
_jalArguments
      Maybe (Maybe RemoteJoins, IntMap Value)
previousStep <- case RemoteJoin
_jalJoin of
        RemoteJoinRemoteSchema RemoteSchemaJoin
remoteSchemaJoin Maybe RemoteJoins
childJoinTree -> do
          let remoteSchemaInfo :: ValidatedRemoteSchemaDef
remoteSchemaInfo = RemoteSchemaInfo -> ValidatedRemoteSchemaDef
rsDef (RemoteSchemaInfo -> ValidatedRemoteSchemaDef)
-> RemoteSchemaInfo -> ValidatedRemoteSchemaDef
forall a b. (a -> b) -> a -> b
$ RemoteSchemaJoin -> RemoteSchemaInfo
_rsjRemoteSchema RemoteSchemaJoin
remoteSchemaJoin
          Maybe (IntMap Value)
maybeJoinIndex <- (GQLReqOutgoing -> m ByteString)
-> UserInfo
-> RemoteSchemaJoin
-> FieldName
-> IntMap JoinArgument
-> m (Maybe (IntMap Value))
forall (m :: * -> *).
(MonadError QErr m, MonadTrace m, MonadIO m) =>
(GQLReqOutgoing -> m ByteString)
-> UserInfo
-> RemoteSchemaJoin
-> FieldName
-> IntMap JoinArgument
-> m (Maybe (IntMap Value))
RS.makeRemoteSchemaJoinCall (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString
callRemoteSchema ValidatedRemoteSchemaDef
remoteSchemaInfo) UserInfo
userInfo RemoteSchemaJoin
remoteSchemaJoin FieldName
_jalFieldName IntMap JoinArgument
joinArguments
          Maybe (Maybe RemoteJoins, IntMap Value)
-> m (Maybe (Maybe RemoteJoins, IntMap Value))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (Maybe RemoteJoins, IntMap Value)
 -> m (Maybe (Maybe RemoteJoins, IntMap Value)))
-> Maybe (Maybe RemoteJoins, IntMap Value)
-> m (Maybe (Maybe RemoteJoins, IntMap Value))
forall a b. (a -> b) -> a -> b
$ (IntMap Value -> (Maybe RemoteJoins, IntMap Value))
-> Maybe (IntMap Value) -> Maybe (Maybe RemoteJoins, IntMap Value)
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Maybe RemoteJoins
childJoinTree,) Maybe (IntMap Value)
maybeJoinIndex
        RemoteJoinSource AnyBackend RemoteSourceJoin
sourceJoin Maybe RemoteJoins
childJoinTree -> do
          Maybe (IntMap Value)
maybeJoinIndex <- (AnyBackend SourceJoinCall -> m ByteString)
-> UserInfo
-> AnyBackend RemoteSourceJoin
-> FieldName
-> IntMap JoinArgument
-> [Header]
-> Maybe Name
-> m (Maybe (IntMap Value))
forall (m :: * -> *).
(MonadQueryTags m, MonadError QErr m, MonadTrace m, MonadIO m) =>
(AnyBackend SourceJoinCall -> m ByteString)
-> UserInfo
-> AnyBackend RemoteSourceJoin
-> FieldName
-> IntMap JoinArgument
-> [Header]
-> Maybe Name
-> m (Maybe (IntMap Value))
S.makeSourceJoinCall AnyBackend SourceJoinCall -> m ByteString
callSource UserInfo
userInfo AnyBackend RemoteSourceJoin
sourceJoin FieldName
_jalFieldName IntMap JoinArgument
joinArguments [Header]
reqHeaders Maybe Name
operationName
          Maybe (Maybe RemoteJoins, IntMap Value)
-> m (Maybe (Maybe RemoteJoins, IntMap Value))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (Maybe RemoteJoins, IntMap Value)
 -> m (Maybe (Maybe RemoteJoins, IntMap Value)))
-> Maybe (Maybe RemoteJoins, IntMap Value)
-> m (Maybe (Maybe RemoteJoins, IntMap Value))
forall a b. (a -> b) -> a -> b
$ (IntMap Value -> (Maybe RemoteJoins, IntMap Value))
-> Maybe (IntMap Value) -> Maybe (Maybe RemoteJoins, IntMap Value)
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (Maybe RemoteJoins
childJoinTree,) Maybe (IntMap Value)
maybeJoinIndex
      Maybe (Maybe RemoteJoins, IntMap Value)
-> ((Maybe RemoteJoins, IntMap Value) -> m (IntMap Value))
-> m (Maybe (IntMap Value))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
t a -> (a -> f b) -> f (t b)
for Maybe (Maybe RemoteJoins, IntMap Value)
previousStep (((Maybe RemoteJoins, IntMap Value) -> m (IntMap Value))
 -> m (Maybe (IntMap Value)))
-> ((Maybe RemoteJoins, IntMap Value) -> m (IntMap Value))
-> m (Maybe (IntMap Value))
forall a b. (a -> b) -> a -> b
$ \(Maybe RemoteJoins
childJoinTree, IntMap Value
joinIndex) -> do
        Maybe RemoteJoins
-> IntMap Value
-> (RemoteJoins -> m (IntMap Value))
-> m (IntMap Value)
forall (f :: * -> *) a.
Applicative f =>
Maybe RemoteJoins -> a -> (RemoteJoins -> f a) -> f a
forRemoteJoins Maybe RemoteJoins
childJoinTree IntMap Value
joinIndex ((RemoteJoins -> m (IntMap Value)) -> m (IntMap Value))
-> (RemoteJoins -> m (IntMap Value)) -> m (IntMap Value)
forall a b. (a -> b) -> a -> b
$ \RemoteJoins
childRemoteJoins -> do
          [Value]
results <-
            (AnyBackend SourceJoinCall -> m ByteString)
-> (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString)
-> UserInfo
-> [Value]
-> RemoteJoins
-> [Header]
-> Maybe Name
-> m [Value]
forall (m :: * -> *) (f :: * -> *).
(MonadError QErr m, MonadQueryTags m, Traversable f, MonadTrace m,
 MonadIO m) =>
(AnyBackend SourceJoinCall -> m ByteString)
-> (ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString)
-> UserInfo
-> f Value
-> RemoteJoins
-> [Header]
-> Maybe Name
-> m (f Value)
foldJoinTreeWith
              AnyBackend SourceJoinCall -> m ByteString
callSource
              ValidatedRemoteSchemaDef -> GQLReqOutgoing -> m ByteString
callRemoteSchema
              UserInfo
userInfo
              (IntMap Value -> [Value]
forall a. IntMap a -> [a]
IntMap.elems IntMap Value
joinIndex)
              RemoteJoins
childRemoteJoins
              [Header]
reqHeaders
              Maybe Name
operationName
          IntMap Value -> m (IntMap Value)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (IntMap Value -> m (IntMap Value))
-> IntMap Value -> m (IntMap Value)
forall a b. (a -> b) -> a -> b
$ [(JoinCallId, Value)] -> IntMap Value
forall a. [(JoinCallId, a)] -> IntMap a
IntMap.fromAscList ([(JoinCallId, Value)] -> IntMap Value)
-> [(JoinCallId, Value)] -> IntMap Value
forall a b. (a -> b) -> a -> b
$ [JoinCallId] -> [Value] -> [(JoinCallId, Value)]
forall a b. [a] -> [b] -> [(a, b)]
zip (IntMap Value -> [JoinCallId]
forall a. IntMap a -> [JoinCallId]
IntMap.keys IntMap Value
joinIndex) [Value]
results
  Text -> m (f Value) -> m (f Value)
forall (m :: * -> *) a.
(MonadIO m, MonadTrace m) =>
Text -> m a -> m a
Tracing.newSpan Text
"Join remote join results"
    (m (f Value) -> m (f Value)) -> m (f Value) -> m (f Value)
forall a b. (a -> b) -> a -> b
$ IntMap (IntMap Value)
-> f (CompositeValue ReplacementToken) -> m (f Value)
forall (f :: * -> *) (m :: * -> *).
(MonadError QErr m, Traversable f) =>
IntMap (IntMap Value)
-> f (CompositeValue ReplacementToken) -> m (f Value)
joinResults IntMap (IntMap Value)
joinIndices f (CompositeValue ReplacementToken)
compositeValue

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

-- | Simple convenient wrapper around @Maybe RemoteJoins@.
forRemoteJoins ::
  (Applicative f) =>
  Maybe RemoteJoins ->
  a ->
  (RemoteJoins -> f a) ->
  f a
forRemoteJoins :: forall (f :: * -> *) a.
Applicative f =>
Maybe RemoteJoins -> a -> (RemoteJoins -> f a) -> f a
forRemoteJoins Maybe RemoteJoins
remoteJoins a
onNoJoins RemoteJoins -> f a
f =
  f a -> (RemoteJoins -> f a) -> Maybe RemoteJoins -> f a
forall b a. b -> (a -> b) -> Maybe a -> b
maybe (a -> f a
forall a. a -> f a
forall (f :: * -> *) a. Applicative f => a -> f a
pure a
onNoJoins) RemoteJoins -> f a
f Maybe RemoteJoins
remoteJoins

-- | When traversing a responses's json, wherever the join columns of a remote
-- join are expected, we want to collect these arguments.
--
-- However looking up by a remote join's definition to collect these arguments
-- does not work because we don't have an 'Ord' or a 'Hashable' instance (it
-- would be a bit of work).
--
-- So this assigned each remote join a unique integer ID by using just the 'Eq'
-- instance. This ID then can be used for the collection of arguments (which
-- should also be faster).
--
-- TODO(nicuveo): https://github.com/hasura/graphql-engine-mono/issues/3891.
assignJoinIds :: JoinTree RemoteJoin -> JoinTree (JoinCallId, RemoteJoin)
assignJoinIds :: RemoteJoins -> JoinTree (JoinCallId, RemoteJoin)
assignJoinIds RemoteJoins
joinTree =
  State
  (JoinCallId, [(JoinCallId, RemoteJoin)])
  (JoinTree (JoinCallId, RemoteJoin))
-> (JoinCallId, [(JoinCallId, RemoteJoin)])
-> JoinTree (JoinCallId, RemoteJoin)
forall s a. State s a -> s -> a
evalState ((RemoteJoin
 -> StateT
      (JoinCallId, [(JoinCallId, RemoteJoin)])
      Identity
      (JoinCallId, RemoteJoin))
-> RemoteJoins
-> State
     (JoinCallId, [(JoinCallId, RemoteJoin)])
     (JoinTree (JoinCallId, RemoteJoin))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> JoinTree a -> f (JoinTree b)
traverse RemoteJoin
-> StateT
     (JoinCallId, [(JoinCallId, RemoteJoin)])
     Identity
     (JoinCallId, RemoteJoin)
assignId RemoteJoins
joinTree) (JoinCallId
0, [])
  where
    assignId ::
      RemoteJoin ->
      State (JoinCallId, [(JoinCallId, RemoteJoin)]) (JoinCallId, RemoteJoin)
    assignId :: RemoteJoin
-> StateT
     (JoinCallId, [(JoinCallId, RemoteJoin)])
     Identity
     (JoinCallId, RemoteJoin)
assignId RemoteJoin
remoteJoin = do
      (JoinCallId
joinCallId, [(JoinCallId, RemoteJoin)]
joinIds) <- StateT
  (JoinCallId, [(JoinCallId, RemoteJoin)])
  Identity
  (JoinCallId, [(JoinCallId, RemoteJoin)])
forall s (m :: * -> *). MonadState s m => m s
get
      let mJoinId :: Maybe (JoinCallId, RemoteJoin)
mJoinId = [(JoinCallId, RemoteJoin)]
joinIds [(JoinCallId, RemoteJoin)]
-> ([(JoinCallId, RemoteJoin)] -> Maybe (JoinCallId, RemoteJoin))
-> Maybe (JoinCallId, RemoteJoin)
forall a b. a -> (a -> b) -> b
& ((JoinCallId, RemoteJoin) -> Bool)
-> [(JoinCallId, RemoteJoin)] -> Maybe (JoinCallId, RemoteJoin)
forall (t :: * -> *) a. Foldable t => (a -> Bool) -> t a -> Maybe a
find \(JoinCallId
_, RemoteJoin
j) -> RemoteJoin
j RemoteJoin -> RemoteJoin -> Bool
forall a. Eq a => a -> a -> Bool
== RemoteJoin
remoteJoin
      Maybe (JoinCallId, RemoteJoin)
mJoinId Maybe (JoinCallId, RemoteJoin)
-> StateT
     (JoinCallId, [(JoinCallId, RemoteJoin)])
     Identity
     (JoinCallId, RemoteJoin)
-> StateT
     (JoinCallId, [(JoinCallId, RemoteJoin)])
     Identity
     (JoinCallId, RemoteJoin)
forall (m :: * -> *) a. Applicative m => Maybe a -> m a -> m a
`onNothing` do
        (JoinCallId, [(JoinCallId, RemoteJoin)])
-> StateT (JoinCallId, [(JoinCallId, RemoteJoin)]) Identity ()
forall s (m :: * -> *). MonadState s m => s -> m ()
put (JoinCallId
joinCallId JoinCallId -> JoinCallId -> JoinCallId
forall a. Num a => a -> a -> a
+ JoinCallId
1, (JoinCallId
joinCallId, RemoteJoin
remoteJoin) (JoinCallId, RemoteJoin)
-> [(JoinCallId, RemoteJoin)] -> [(JoinCallId, RemoteJoin)]
forall a. a -> [a] -> [a]
: [(JoinCallId, RemoteJoin)]
joinIds)
        (JoinCallId, RemoteJoin)
-> StateT
     (JoinCallId, [(JoinCallId, RemoteJoin)])
     Identity
     (JoinCallId, RemoteJoin)
forall a.
a -> StateT (JoinCallId, [(JoinCallId, RemoteJoin)]) Identity a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (JoinCallId
joinCallId, RemoteJoin
remoteJoin)

collectJoinArguments ::
  forall f m.
  (MonadError QErr m, Traversable f) =>
  JoinTree (JoinCallId, RemoteJoin) ->
  f JO.Value ->
  m (f (CompositeValue ReplacementToken), IntMap.IntMap JoinArguments)
collectJoinArguments :: forall (f :: * -> *) (m :: * -> *).
(MonadError QErr m, Traversable f) =>
JoinTree (JoinCallId, RemoteJoin)
-> f Value
-> m (f (CompositeValue ReplacementToken), IntMap JoinArguments)
collectJoinArguments JoinTree (JoinCallId, RemoteJoin)
joinTree f Value
lhs = do
  (f (CompositeValue ReplacementToken),
 (JoinCallId, IntMap JoinArguments))
result <- (StateT
   (JoinCallId, IntMap JoinArguments)
   m
   (f (CompositeValue ReplacementToken))
 -> (JoinCallId, IntMap JoinArguments)
 -> m (f (CompositeValue ReplacementToken),
       (JoinCallId, IntMap JoinArguments)))
-> (JoinCallId, IntMap JoinArguments)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (f (CompositeValue ReplacementToken))
-> m (f (CompositeValue ReplacementToken),
      (JoinCallId, IntMap JoinArguments))
forall a b c. (a -> b -> c) -> b -> a -> c
flip StateT
  (JoinCallId, IntMap JoinArguments)
  m
  (f (CompositeValue ReplacementToken))
-> (JoinCallId, IntMap JoinArguments)
-> m (f (CompositeValue ReplacementToken),
      (JoinCallId, IntMap JoinArguments))
forall s (m :: * -> *) a. StateT s m a -> s -> m (a, s)
runStateT (JoinCallId
0, IntMap JoinArguments
forall a. Monoid a => a
mempty) (StateT
   (JoinCallId, IntMap JoinArguments)
   m
   (f (CompositeValue ReplacementToken))
 -> m (f (CompositeValue ReplacementToken),
       (JoinCallId, IntMap JoinArguments)))
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (f (CompositeValue ReplacementToken))
-> m (f (CompositeValue ReplacementToken),
      (JoinCallId, IntMap JoinArguments))
forall a b. (a -> b) -> a -> b
$ (Value
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (CompositeValue ReplacementToken))
-> f Value
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (f (CompositeValue ReplacementToken))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> f a -> f (f b)
traverse (JoinTree (JoinCallId, RemoteJoin)
-> Value
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
traverseValue JoinTree (JoinCallId, RemoteJoin)
joinTree) f Value
lhs
  -- Discard the 'JoinArgumentId' from the intermediate state transformation.
  (f (CompositeValue ReplacementToken), IntMap JoinArguments)
-> m (f (CompositeValue ReplacementToken), IntMap JoinArguments)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ((f (CompositeValue ReplacementToken), IntMap JoinArguments)
 -> m (f (CompositeValue ReplacementToken), IntMap JoinArguments))
-> (f (CompositeValue ReplacementToken), IntMap JoinArguments)
-> m (f (CompositeValue ReplacementToken), IntMap JoinArguments)
forall a b. (a -> b) -> a -> b
$ ((JoinCallId, IntMap JoinArguments) -> IntMap JoinArguments)
-> (f (CompositeValue ReplacementToken),
    (JoinCallId, IntMap JoinArguments))
-> (f (CompositeValue ReplacementToken), IntMap JoinArguments)
forall b c d. (b -> c) -> (d, b) -> (d, c)
forall (a :: * -> * -> *) b c d.
Arrow a =>
a b c -> a (d, b) (d, c)
second (JoinCallId, IntMap JoinArguments) -> IntMap JoinArguments
forall a b. (a, b) -> b
snd (f (CompositeValue ReplacementToken),
 (JoinCallId, IntMap JoinArguments))
result
  where
    getReplacementToken ::
      IntMap.Key ->
      RemoteJoin ->
      JoinArgument ->
      FieldName ->
      StateT
        (JoinArgumentId, IntMap.IntMap JoinArguments)
        m
        ReplacementToken
    getReplacementToken :: JoinCallId
-> RemoteJoin
-> JoinArgument
-> FieldName
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
getReplacementToken JoinCallId
joinId RemoteJoin
remoteJoin JoinArgument
argument FieldName
fieldName = do
      (JoinCallId
counter, IntMap JoinArguments
joins) <- StateT
  (JoinCallId, IntMap JoinArguments)
  m
  (JoinCallId, IntMap JoinArguments)
forall s (m :: * -> *). MonadState s m => m s
get
      case JoinCallId -> IntMap JoinArguments -> Maybe JoinArguments
forall a. JoinCallId -> IntMap a -> Maybe a
IntMap.lookup JoinCallId
joinId IntMap JoinArguments
joins of
        -- XXX: We're making an explicit decision to ignore the existing
        -- 'fieldName' and replace it with the argument provided to this
        -- function.
        --
        -- This needs to be tested so we can verify that the result of this
        -- function call is reasonable.
        Just (JoinArguments RemoteJoin
_remoteJoin HashMap JoinArgument JoinCallId
arguments FieldName
_fieldName) ->
          case JoinArgument -> HashMap JoinArgument JoinCallId -> Maybe JoinCallId
forall k v. (Eq k, Hashable k) => k -> HashMap k v -> Maybe v
HashMap.lookup JoinArgument
argument HashMap JoinArgument JoinCallId
arguments of
            Just JoinCallId
argumentId -> ReplacementToken
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ReplacementToken
 -> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken)
-> ReplacementToken
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
forall a b. (a -> b) -> a -> b
$ JoinCallId -> JoinCallId -> ReplacementToken
ReplacementToken JoinCallId
joinId JoinCallId
argumentId
            Maybe JoinCallId
Nothing -> JoinCallId
-> IntMap JoinArguments
-> HashMap JoinArgument JoinCallId
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
addNewArgument JoinCallId
counter IntMap JoinArguments
joins HashMap JoinArgument JoinCallId
arguments
        Maybe JoinArguments
Nothing -> JoinCallId
-> IntMap JoinArguments
-> HashMap JoinArgument JoinCallId
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
addNewArgument JoinCallId
counter IntMap JoinArguments
joins HashMap JoinArgument JoinCallId
forall a. Monoid a => a
mempty
      where
        addNewArgument :: JoinCallId
-> IntMap JoinArguments
-> HashMap JoinArgument JoinCallId
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
addNewArgument JoinCallId
counter IntMap JoinArguments
joins HashMap JoinArgument JoinCallId
arguments = do
          let argumentId :: JoinCallId
argumentId = JoinCallId
counter
              newArguments :: JoinArguments
newArguments =
                RemoteJoin
-> HashMap JoinArgument JoinCallId -> FieldName -> JoinArguments
JoinArguments
                  RemoteJoin
remoteJoin
                  (JoinArgument
-> JoinCallId
-> HashMap JoinArgument JoinCallId
-> HashMap JoinArgument JoinCallId
forall k v.
(Eq k, Hashable k) =>
k -> v -> HashMap k v -> HashMap k v
HashMap.insert JoinArgument
argument JoinCallId
argumentId HashMap JoinArgument JoinCallId
arguments)
                  FieldName
fieldName
          (JoinCallId, IntMap JoinArguments)
-> StateT (JoinCallId, IntMap JoinArguments) m ()
forall s (m :: * -> *). MonadState s m => s -> m ()
put (JoinCallId
counter JoinCallId -> JoinCallId -> JoinCallId
forall a. Num a => a -> a -> a
+ JoinCallId
1, JoinCallId
-> JoinArguments -> IntMap JoinArguments -> IntMap JoinArguments
forall a. JoinCallId -> a -> IntMap a -> IntMap a
IntMap.insert JoinCallId
joinId JoinArguments
newArguments IntMap JoinArguments
joins)
          ReplacementToken
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (ReplacementToken
 -> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken)
-> ReplacementToken
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
forall a b. (a -> b) -> a -> b
$ JoinCallId -> JoinCallId -> ReplacementToken
ReplacementToken JoinCallId
joinId JoinCallId
argumentId

    traverseValue ::
      JoinTree (IntMap.Key, RemoteJoin) ->
      JO.Value ->
      StateT
        (JoinArgumentId, IntMap.IntMap JoinArguments)
        m
        (CompositeValue ReplacementToken)
    traverseValue :: JoinTree (JoinCallId, RemoteJoin)
-> Value
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
traverseValue JoinTree (JoinCallId, RemoteJoin)
joinTree_ = \case
      -- 'JO.Null' is a special case of scalar value here, which indicates that
      -- the previous step did not return enough data for us to continue
      -- traversing down this path.
      --
      -- This can occur in the following cases:
      --  * Permission errors; when the user joins on a value they are not
      --    allowed to access
      --  * Queries with remote sources that resolve to null, for example:
      -- {
      --    q {
      --      user_by_pk() {
      --        id
      --        name
      --        r {
      --        }
      --        address {
      --          r_geo {
      --          }
      --        }
      --      }
      --    }
      -- }
      Value
JO.Null -> CompositeValue ReplacementToken
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (CompositeValue ReplacementToken
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (CompositeValue ReplacementToken))
-> CompositeValue ReplacementToken
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
forall a b. (a -> b) -> a -> b
$ Value -> CompositeValue ReplacementToken
forall a. Value -> CompositeValue a
CVOrdValue Value
JO.Null
      JO.Object Object
object -> CompositeObject ReplacementToken -> CompositeValue ReplacementToken
forall a. CompositeObject a -> CompositeValue a
CVObject (CompositeObject ReplacementToken
 -> CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeObject ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> JoinTree (JoinCallId, RemoteJoin)
-> Object
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeObject ReplacementToken)
traverseObject JoinTree (JoinCallId, RemoteJoin)
joinTree_ Object
object
      JO.Array Array
array -> [CompositeValue ReplacementToken]
-> CompositeValue ReplacementToken
forall a. [CompositeValue a] -> CompositeValue a
CVObjectArray ([CompositeValue ReplacementToken]
 -> CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     [CompositeValue ReplacementToken]
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (Value
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (CompositeValue ReplacementToken))
-> [Value]
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     [CompositeValue ReplacementToken]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM (JoinTree (JoinCallId, RemoteJoin)
-> Value
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
traverseValue JoinTree (JoinCallId, RemoteJoin)
joinTree_) (Array -> [Value]
forall a. Vector a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList Array
array)
      Value
_ -> Text
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 Text
"found a scalar value when traversing with a non-empty join tree"

    traverseObject ::
      JoinTree (IntMap.Key, RemoteJoin) ->
      JO.Object ->
      StateT
        (JoinArgumentId, IntMap.IntMap JoinArguments)
        m
        (InsOrdHashMap Text (CompositeValue ReplacementToken))
    traverseObject :: JoinTree (JoinCallId, RemoteJoin)
-> Object
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeObject ReplacementToken)
traverseObject JoinTree (JoinCallId, RemoteJoin)
joinTree_ Object
object = do
      let joinTreeNodes :: NEHashMap QualifiedFieldName (JoinNode (JoinCallId, RemoteJoin))
joinTreeNodes = JoinTree (JoinCallId, RemoteJoin)
-> NEHashMap QualifiedFieldName (JoinNode (JoinCallId, RemoteJoin))
forall a. JoinTree a -> NEHashMap QualifiedFieldName (JoinNode a)
unJoinTree JoinTree (JoinCallId, RemoteJoin)
joinTree_
          phantomFields :: HashSet Text
phantomFields =
            [Text] -> HashSet Text
forall a. (Eq a, Hashable a) => [a] -> HashSet a
HS.fromList
              ([Text] -> HashSet Text) -> [Text] -> HashSet Text
forall a b. (a -> b) -> a -> b
$ (FieldName -> Text) -> [FieldName] -> [Text]
forall a b. (a -> b) -> [a] -> [b]
map FieldName -> Text
getFieldNameTxt
              ([FieldName] -> [Text]) -> [FieldName] -> [Text]
forall a b. (a -> b) -> a -> b
$ ((JoinCallId, RemoteJoin) -> [FieldName])
-> [(JoinCallId, RemoteJoin)] -> [FieldName]
forall (t :: * -> *) a b. Foldable t => (a -> [b]) -> t a -> [b]
concatMap (RemoteJoin -> [FieldName]
getPhantomFields (RemoteJoin -> [FieldName])
-> ((JoinCallId, RemoteJoin) -> RemoteJoin)
-> (JoinCallId, RemoteJoin)
-> [FieldName]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (JoinCallId, RemoteJoin) -> RemoteJoin
forall a b. (a, b) -> b
snd)
              ([(JoinCallId, RemoteJoin)] -> [FieldName])
-> [(JoinCallId, RemoteJoin)] -> [FieldName]
forall a b. (a -> b) -> a -> b
$ JoinTree (JoinCallId, RemoteJoin) -> [(JoinCallId, RemoteJoin)]
forall a. JoinTree a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList JoinTree (JoinCallId, RemoteJoin)
joinTree_

      -- If we need the typename to disambiguate branches in the join tree, it
      -- will be present in the answer as a placeholder internal field.
      --
      -- We currently have no way of checking whether we explicitly requested
      -- that field, and it would be possible for a malicious user to attempt to
      -- spoof that value by explicitly requesting a value they control.
      -- However, there's no actual risk: we only use that value for lookups
      -- inside the join tree, and if we didn't request this field, the keys in
      -- the join tree map will explicitly require a typename NOT to be
      -- provided. Meaning that any spoofing attempt will just, at worst, result
      -- in remote joins not being performed.
      --
      -- We always remove that key from the resulting object.
      Maybe Text
joinTypeName <- case Text -> Object -> Maybe Value
JO.lookup Text
"__hasura_internal_typename" Object
object of
        Maybe Value
Nothing -> Maybe Text
-> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text)
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe Text
forall a. Maybe a
Nothing
        Just (JO.String Text
typename) -> Maybe Text
-> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text)
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe Text
 -> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text))
-> Maybe Text
-> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text)
forall a b. (a -> b) -> a -> b
$ Text -> Maybe Text
forall a. a -> Maybe a
Just Text
typename
        Just Value
value -> Text -> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text)
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 (Text -> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text))
-> Text -> StateT (JoinCallId, IntMap JoinArguments) m (Maybe Text)
forall a b. (a -> b) -> a -> b
$ Text
"The reserved __hasura_internal_typename field contains an unexpected value: " Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Value -> Text
forall a. Show a => a -> Text
tshow Value
value

      -- during this traversal we assume that the remote join column has some
      -- placeholder value in the response. If this weren't present it would
      -- involve a lot more book-keeping to preserve the order of the original
      -- selection set in the response
      [(Text, Maybe (CompositeValue ReplacementToken))]
compositeObject <- [(Text, Value)]
-> ((Text, Value)
    -> StateT
         (JoinCallId, IntMap JoinArguments)
         m
         (Text, Maybe (CompositeValue ReplacementToken)))
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     [(Text, Maybe (CompositeValue ReplacementToken))]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
t a -> (a -> f b) -> f (t b)
for (Object -> [(Text, Value)]
JO.toList Object
object) (((Text, Value)
  -> StateT
       (JoinCallId, IntMap JoinArguments)
       m
       (Text, Maybe (CompositeValue ReplacementToken)))
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      [(Text, Maybe (CompositeValue ReplacementToken))])
-> ((Text, Value)
    -> StateT
         (JoinCallId, IntMap JoinArguments)
         m
         (Text, Maybe (CompositeValue ReplacementToken)))
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     [(Text, Maybe (CompositeValue ReplacementToken))]
forall a b. (a -> b) -> a -> b
$ \(Text
fieldName, Value
value_) ->
        (Text
fieldName,) (Maybe (CompositeValue ReplacementToken)
 -> (Text, Maybe (CompositeValue ReplacementToken)))
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Text, Maybe (CompositeValue ReplacementToken))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> case QualifiedFieldName
-> NEHashMap QualifiedFieldName (JoinNode (JoinCallId, RemoteJoin))
-> Maybe (JoinNode (JoinCallId, RemoteJoin))
forall k v. Hashable k => k -> NEHashMap k v -> Maybe v
NEMap.lookup (Maybe Text -> Text -> QualifiedFieldName
QualifiedFieldName Maybe Text
joinTypeName Text
fieldName) NEHashMap QualifiedFieldName (JoinNode (JoinCallId, RemoteJoin))
joinTreeNodes of
          Just (Leaf (JoinCallId
joinId, RemoteJoin
remoteJoin)) -> do
            HashMap FieldName Value
joinArgument <- HashMap FieldName JoinColumnAlias
-> (JoinColumnAlias
    -> StateT (JoinCallId, IntMap JoinArguments) m Value)
-> StateT
     (JoinCallId, IntMap JoinArguments) m (HashMap FieldName Value)
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
t a -> (a -> m b) -> m (t b)
forM (RemoteJoin -> HashMap FieldName JoinColumnAlias
getJoinColumnMapping RemoteJoin
remoteJoin) ((JoinColumnAlias
  -> StateT (JoinCallId, IntMap JoinArguments) m Value)
 -> StateT
      (JoinCallId, IntMap JoinArguments) m (HashMap FieldName Value))
-> (JoinColumnAlias
    -> StateT (JoinCallId, IntMap JoinArguments) m Value)
-> StateT
     (JoinCallId, IntMap JoinArguments) m (HashMap FieldName Value)
forall a b. (a -> b) -> a -> b
$ \JoinColumnAlias
alias -> do
              let aliasTxt :: Text
aliasTxt = FieldName -> Text
getFieldNameTxt (FieldName -> Text) -> FieldName -> Text
forall a b. (a -> b) -> a -> b
$ JoinColumnAlias -> FieldName
getAliasFieldName JoinColumnAlias
alias
              Maybe Value
-> StateT (JoinCallId, IntMap JoinArguments) m Value
-> StateT (JoinCallId, IntMap JoinArguments) m Value
forall (m :: * -> *) a. Applicative m => Maybe a -> m a -> m a
onNothing (Text -> Object -> Maybe Value
JO.lookup Text
aliasTxt Object
object)
                (StateT (JoinCallId, IntMap JoinArguments) m Value
 -> StateT (JoinCallId, IntMap JoinArguments) m Value)
-> StateT (JoinCallId, IntMap JoinArguments) m Value
-> StateT (JoinCallId, IntMap JoinArguments) m Value
forall a b. (a -> b) -> a -> b
$ Text -> StateT (JoinCallId, IntMap JoinArguments) m Value
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500
                (Text -> StateT (JoinCallId, IntMap JoinArguments) m Value)
-> Text -> StateT (JoinCallId, IntMap JoinArguments) m Value
forall a b. (a -> b) -> a -> b
$ Text
"a join column is missing from the response: "
                Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
aliasTxt
            if HashMap FieldName Value -> Bool
forall k v. HashMap k v -> Bool
HashMap.null ((Value -> Bool)
-> HashMap FieldName Value -> HashMap FieldName Value
forall v k. (v -> Bool) -> HashMap k v -> HashMap k v
HashMap.filter (Value -> Value -> Bool
forall a. Eq a => a -> a -> Bool
== Value
JO.Null) HashMap FieldName Value
joinArgument)
              then
                CompositeValue ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall a. a -> Maybe a
Just
                  (CompositeValue ReplacementToken
 -> Maybe (CompositeValue ReplacementToken))
-> (ReplacementToken -> CompositeValue ReplacementToken)
-> ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ReplacementToken -> CompositeValue ReplacementToken
forall a. a -> CompositeValue a
CVFromRemote
                  (ReplacementToken -> Maybe (CompositeValue ReplacementToken))
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> JoinCallId
-> RemoteJoin
-> JoinArgument
-> FieldName
-> StateT (JoinCallId, IntMap JoinArguments) m ReplacementToken
getReplacementToken JoinCallId
joinId RemoteJoin
remoteJoin (HashMap FieldName Value -> JoinArgument
JoinArgument HashMap FieldName Value
joinArgument) (Text -> FieldName
FieldName Text
fieldName)
              else -- we do not join with the remote field if any of the leaves of
              -- the join argument are null
                Maybe (CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (CompositeValue ReplacementToken)
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (Maybe (CompositeValue ReplacementToken)))
-> Maybe (CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall a b. (a -> b) -> a -> b
$ CompositeValue ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall a. a -> Maybe a
Just (CompositeValue ReplacementToken
 -> Maybe (CompositeValue ReplacementToken))
-> CompositeValue ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall a b. (a -> b) -> a -> b
$ Value -> CompositeValue ReplacementToken
forall a. Value -> CompositeValue a
CVOrdValue Value
JO.Null
          Just (Tree JoinTree (JoinCallId, RemoteJoin)
joinSubTree) ->
            CompositeValue ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall a. a -> Maybe a
Just (CompositeValue ReplacementToken
 -> Maybe (CompositeValue ReplacementToken))
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> JoinTree (JoinCallId, RemoteJoin)
-> Value
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeValue ReplacementToken)
traverseValue JoinTree (JoinCallId, RemoteJoin)
joinSubTree Value
value_
          Maybe (JoinNode (JoinCallId, RemoteJoin))
Nothing ->
            if Text -> HashSet Text -> Bool
forall a. (Eq a, Hashable a) => a -> HashSet a -> Bool
HS.member Text
fieldName HashSet Text
phantomFields Bool -> Bool -> Bool
|| Text
fieldName Text -> Text -> Bool
forall a. Eq a => a -> a -> Bool
== Text
"__hasura_internal_typename"
              then Maybe (CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Maybe (CompositeValue ReplacementToken)
forall a. Maybe a
Nothing
              else Maybe (CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Maybe (CompositeValue ReplacementToken)
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (Maybe (CompositeValue ReplacementToken)))
-> Maybe (CompositeValue ReplacementToken)
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (Maybe (CompositeValue ReplacementToken))
forall a b. (a -> b) -> a -> b
$ CompositeValue ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall a. a -> Maybe a
Just (CompositeValue ReplacementToken
 -> Maybe (CompositeValue ReplacementToken))
-> CompositeValue ReplacementToken
-> Maybe (CompositeValue ReplacementToken)
forall a b. (a -> b) -> a -> b
$ Value -> CompositeValue ReplacementToken
forall a. Value -> CompositeValue a
CVOrdValue Value
value_

      CompositeObject ReplacementToken
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeObject ReplacementToken)
forall a. a -> StateT (JoinCallId, IntMap JoinArguments) m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        (CompositeObject ReplacementToken
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (CompositeObject ReplacementToken))
-> ([(Text, CompositeValue ReplacementToken)]
    -> CompositeObject ReplacementToken)
-> [(Text, CompositeValue ReplacementToken)]
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeObject ReplacementToken)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. [(Text, CompositeValue ReplacementToken)]
-> CompositeObject ReplacementToken
forall k v. (Eq k, Hashable k) => [(k, v)] -> InsOrdHashMap k v
InsOrdHashMap.fromList
        ([(Text, CompositeValue ReplacementToken)]
 -> StateT
      (JoinCallId, IntMap JoinArguments)
      m
      (CompositeObject ReplacementToken))
-> [(Text, CompositeValue ReplacementToken)]
-> StateT
     (JoinCallId, IntMap JoinArguments)
     m
     (CompositeObject ReplacementToken)
forall a b. (a -> b) -> a -> b
$
        -- filter out the Nothings
        ((Text, Maybe (CompositeValue ReplacementToken))
 -> Maybe (Text, CompositeValue ReplacementToken))
-> [(Text, Maybe (CompositeValue ReplacementToken))]
-> [(Text, CompositeValue ReplacementToken)]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe (Text, Maybe (CompositeValue ReplacementToken))
-> Maybe (Text, CompositeValue ReplacementToken)
forall (t :: * -> *) (f :: * -> *) a.
(Traversable t, Applicative f) =>
t (f a) -> f (t a)
forall (f :: * -> *) a. Applicative f => (Text, f a) -> f (Text, a)
sequenceA [(Text, Maybe (CompositeValue ReplacementToken))]
compositeObject

joinResults ::
  forall f m.
  (MonadError QErr m, Traversable f) =>
  IntMap.IntMap (IntMap.IntMap JO.Value) ->
  f (CompositeValue ReplacementToken) ->
  m (f JO.Value)
joinResults :: forall (f :: * -> *) (m :: * -> *).
(MonadError QErr m, Traversable f) =>
IntMap (IntMap Value)
-> f (CompositeValue ReplacementToken) -> m (f Value)
joinResults IntMap (IntMap Value)
remoteResults f (CompositeValue ReplacementToken)
compositeValues = do
  (CompositeValue ReplacementToken -> m Value)
-> f (CompositeValue ReplacementToken) -> m (f Value)
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> f a -> f (f b)
traverse ((CompositeValue Value -> Value)
-> m (CompositeValue Value) -> m Value
forall a b. (a -> b) -> m a -> m b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap CompositeValue Value -> Value
compositeValueToJSON (m (CompositeValue Value) -> m Value)
-> (CompositeValue ReplacementToken -> m (CompositeValue Value))
-> CompositeValue ReplacementToken
-> m Value
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ReplacementToken -> m Value)
-> CompositeValue ReplacementToken -> m (CompositeValue Value)
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> CompositeValue a -> f (CompositeValue b)
traverse ReplacementToken -> m Value
replaceToken) f (CompositeValue ReplacementToken)
compositeValues
  where
    replaceToken :: ReplacementToken -> m JO.Value
    replaceToken :: ReplacementToken -> m Value
replaceToken (ReplacementToken JoinCallId
joinCallId JoinCallId
argumentId) = do
      IntMap Value
joinCallResults <-
        Maybe (IntMap Value) -> m (IntMap Value) -> m (IntMap Value)
forall (m :: * -> *) a. Applicative m => Maybe a -> m a -> m a
onNothing (JoinCallId -> IntMap (IntMap Value) -> Maybe (IntMap Value)
forall a. JoinCallId -> IntMap a -> Maybe a
IntMap.lookup JoinCallId
joinCallId IntMap (IntMap Value)
remoteResults)
          (m (IntMap Value) -> m (IntMap Value))
-> m (IntMap Value) -> m (IntMap Value)
forall a b. (a -> b) -> a -> b
$ Text -> m (IntMap Value)
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500
          (Text -> m (IntMap Value)) -> Text -> m (IntMap Value)
forall a b. (a -> b) -> a -> b
$ Text
"couldn't find results for the join with id: "
          Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> JoinCallId -> Text
forall a. Show a => a -> Text
tshow JoinCallId
joinCallId
      Maybe Value -> m Value -> m Value
forall (m :: * -> *) a. Applicative m => Maybe a -> m a -> m a
onNothing (JoinCallId -> IntMap Value -> Maybe Value
forall a. JoinCallId -> IntMap a -> Maybe a
IntMap.lookup JoinCallId
argumentId IntMap Value
joinCallResults)
        (m Value -> m Value) -> m Value -> m Value
forall a b. (a -> b) -> a -> b
$ Text -> m Value
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500
        (Text -> m Value) -> Text -> m Value
forall a b. (a -> b) -> a -> b
$ Text
"couldn't find a value for argument id in the join results: "
        Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> (JoinCallId, JoinCallId) -> Text
forall a. Show a => a -> Text
tshow (JoinCallId
argumentId, JoinCallId
joinCallId)

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

type CompositeObject a = InsOrdHashMap.InsOrdHashMap Text (CompositeValue a)

-- | A hybrid JSON value representation which captures the context of remote join field in type parameter.
data CompositeValue a
  = CVOrdValue !JO.Value
  | CVObject !(CompositeObject a)
  | CVObjectArray ![CompositeValue a]
  | CVFromRemote !a
  deriving (JoinCallId -> CompositeValue a -> ShowS
[CompositeValue a] -> ShowS
CompositeValue a -> String
(JoinCallId -> CompositeValue a -> ShowS)
-> (CompositeValue a -> String)
-> ([CompositeValue a] -> ShowS)
-> Show (CompositeValue a)
forall a. Show a => JoinCallId -> CompositeValue a -> ShowS
forall a. Show a => [CompositeValue a] -> ShowS
forall a. Show a => CompositeValue a -> String
forall a.
(JoinCallId -> a -> ShowS)
-> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: forall a. Show a => JoinCallId -> CompositeValue a -> ShowS
showsPrec :: JoinCallId -> CompositeValue a -> ShowS
$cshow :: forall a. Show a => CompositeValue a -> String
show :: CompositeValue a -> String
$cshowList :: forall a. Show a => [CompositeValue a] -> ShowS
showList :: [CompositeValue a] -> ShowS
Show, CompositeValue a -> CompositeValue a -> Bool
(CompositeValue a -> CompositeValue a -> Bool)
-> (CompositeValue a -> CompositeValue a -> Bool)
-> Eq (CompositeValue a)
forall a. Eq a => CompositeValue a -> CompositeValue a -> Bool
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: forall a. Eq a => CompositeValue a -> CompositeValue a -> Bool
== :: CompositeValue a -> CompositeValue a -> Bool
$c/= :: forall a. Eq a => CompositeValue a -> CompositeValue a -> Bool
/= :: CompositeValue a -> CompositeValue a -> Bool
Eq, (forall a b. (a -> b) -> CompositeValue a -> CompositeValue b)
-> (forall a b. a -> CompositeValue b -> CompositeValue a)
-> Functor CompositeValue
forall a b. a -> CompositeValue b -> CompositeValue a
forall a b. (a -> b) -> CompositeValue a -> CompositeValue b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall a b. (a -> b) -> CompositeValue a -> CompositeValue b
fmap :: forall a b. (a -> b) -> CompositeValue a -> CompositeValue b
$c<$ :: forall a b. a -> CompositeValue b -> CompositeValue a
<$ :: forall a b. a -> CompositeValue b -> CompositeValue a
Functor, (forall m. Monoid m => CompositeValue m -> m)
-> (forall m a. Monoid m => (a -> m) -> CompositeValue a -> m)
-> (forall m a. Monoid m => (a -> m) -> CompositeValue a -> m)
-> (forall a b. (a -> b -> b) -> b -> CompositeValue a -> b)
-> (forall a b. (a -> b -> b) -> b -> CompositeValue a -> b)
-> (forall b a. (b -> a -> b) -> b -> CompositeValue a -> b)
-> (forall b a. (b -> a -> b) -> b -> CompositeValue a -> b)
-> (forall a. (a -> a -> a) -> CompositeValue a -> a)
-> (forall a. (a -> a -> a) -> CompositeValue a -> a)
-> (forall a. CompositeValue a -> [a])
-> (forall a. CompositeValue a -> Bool)
-> (forall a. CompositeValue a -> JoinCallId)
-> (forall a. Eq a => a -> CompositeValue a -> Bool)
-> (forall a. Ord a => CompositeValue a -> a)
-> (forall a. Ord a => CompositeValue a -> a)
-> (forall a. Num a => CompositeValue a -> a)
-> (forall a. Num a => CompositeValue a -> a)
-> Foldable CompositeValue
forall a. Eq a => a -> CompositeValue a -> Bool
forall a. Num a => CompositeValue a -> a
forall a. Ord a => CompositeValue a -> a
forall m. Monoid m => CompositeValue m -> m
forall a. CompositeValue a -> Bool
forall a. CompositeValue a -> JoinCallId
forall a. CompositeValue a -> [a]
forall a. (a -> a -> a) -> CompositeValue a -> a
forall m a. Monoid m => (a -> m) -> CompositeValue a -> m
forall b a. (b -> a -> b) -> b -> CompositeValue a -> b
forall a b. (a -> b -> b) -> b -> CompositeValue a -> b
forall (t :: * -> *).
(forall m. Monoid m => t m -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall m a. Monoid m => (a -> m) -> t a -> m)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall a b. (a -> b -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall b a. (b -> a -> b) -> b -> t a -> b)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. (a -> a -> a) -> t a -> a)
-> (forall a. t a -> [a])
-> (forall a. t a -> Bool)
-> (forall a. t a -> JoinCallId)
-> (forall a. Eq a => a -> t a -> Bool)
-> (forall a. Ord a => t a -> a)
-> (forall a. Ord a => t a -> a)
-> (forall a. Num a => t a -> a)
-> (forall a. Num a => t a -> a)
-> Foldable t
$cfold :: forall m. Monoid m => CompositeValue m -> m
fold :: forall m. Monoid m => CompositeValue m -> m
$cfoldMap :: forall m a. Monoid m => (a -> m) -> CompositeValue a -> m
foldMap :: forall m a. Monoid m => (a -> m) -> CompositeValue a -> m
$cfoldMap' :: forall m a. Monoid m => (a -> m) -> CompositeValue a -> m
foldMap' :: forall m a. Monoid m => (a -> m) -> CompositeValue a -> m
$cfoldr :: forall a b. (a -> b -> b) -> b -> CompositeValue a -> b
foldr :: forall a b. (a -> b -> b) -> b -> CompositeValue a -> b
$cfoldr' :: forall a b. (a -> b -> b) -> b -> CompositeValue a -> b
foldr' :: forall a b. (a -> b -> b) -> b -> CompositeValue a -> b
$cfoldl :: forall b a. (b -> a -> b) -> b -> CompositeValue a -> b
foldl :: forall b a. (b -> a -> b) -> b -> CompositeValue a -> b
$cfoldl' :: forall b a. (b -> a -> b) -> b -> CompositeValue a -> b
foldl' :: forall b a. (b -> a -> b) -> b -> CompositeValue a -> b
$cfoldr1 :: forall a. (a -> a -> a) -> CompositeValue a -> a
foldr1 :: forall a. (a -> a -> a) -> CompositeValue a -> a
$cfoldl1 :: forall a. (a -> a -> a) -> CompositeValue a -> a
foldl1 :: forall a. (a -> a -> a) -> CompositeValue a -> a
$ctoList :: forall a. CompositeValue a -> [a]
toList :: forall a. CompositeValue a -> [a]
$cnull :: forall a. CompositeValue a -> Bool
null :: forall a. CompositeValue a -> Bool
$clength :: forall a. CompositeValue a -> JoinCallId
length :: forall a. CompositeValue a -> JoinCallId
$celem :: forall a. Eq a => a -> CompositeValue a -> Bool
elem :: forall a. Eq a => a -> CompositeValue a -> Bool
$cmaximum :: forall a. Ord a => CompositeValue a -> a
maximum :: forall a. Ord a => CompositeValue a -> a
$cminimum :: forall a. Ord a => CompositeValue a -> a
minimum :: forall a. Ord a => CompositeValue a -> a
$csum :: forall a. Num a => CompositeValue a -> a
sum :: forall a. Num a => CompositeValue a -> a
$cproduct :: forall a. Num a => CompositeValue a -> a
product :: forall a. Num a => CompositeValue a -> a
Foldable, Functor CompositeValue
Foldable CompositeValue
Functor CompositeValue
-> Foldable CompositeValue
-> (forall (f :: * -> *) a b.
    Applicative f =>
    (a -> f b) -> CompositeValue a -> f (CompositeValue b))
-> (forall (f :: * -> *) a.
    Applicative f =>
    CompositeValue (f a) -> f (CompositeValue a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> CompositeValue a -> m (CompositeValue b))
-> (forall (m :: * -> *) a.
    Monad m =>
    CompositeValue (m a) -> m (CompositeValue a))
-> Traversable CompositeValue
forall (t :: * -> *).
Functor t
-> Foldable t
-> (forall (f :: * -> *) a b.
    Applicative f =>
    (a -> f b) -> t a -> f (t b))
-> (forall (f :: * -> *) a. Applicative f => t (f a) -> f (t a))
-> (forall (m :: * -> *) a b.
    Monad m =>
    (a -> m b) -> t a -> m (t b))
-> (forall (m :: * -> *) a. Monad m => t (m a) -> m (t a))
-> Traversable t
forall (m :: * -> *) a.
Monad m =>
CompositeValue (m a) -> m (CompositeValue a)
forall (f :: * -> *) a.
Applicative f =>
CompositeValue (f a) -> f (CompositeValue a)
forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> CompositeValue a -> m (CompositeValue b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> CompositeValue a -> f (CompositeValue b)
$ctraverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> CompositeValue a -> f (CompositeValue b)
traverse :: forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> CompositeValue a -> f (CompositeValue b)
$csequenceA :: forall (f :: * -> *) a.
Applicative f =>
CompositeValue (f a) -> f (CompositeValue a)
sequenceA :: forall (f :: * -> *) a.
Applicative f =>
CompositeValue (f a) -> f (CompositeValue a)
$cmapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> CompositeValue a -> m (CompositeValue b)
mapM :: forall (m :: * -> *) a b.
Monad m =>
(a -> m b) -> CompositeValue a -> m (CompositeValue b)
$csequence :: forall (m :: * -> *) a.
Monad m =>
CompositeValue (m a) -> m (CompositeValue a)
sequence :: forall (m :: * -> *) a.
Monad m =>
CompositeValue (m a) -> m (CompositeValue a)
Traversable)

compositeValueToJSON :: CompositeValue JO.Value -> JO.Value
compositeValueToJSON :: CompositeValue Value -> Value
compositeValueToJSON = \case
  CVOrdValue Value
v -> Value
v
  CVObject CompositeObject Value
obj -> [(Text, Value)] -> Value
JO.object ([(Text, Value)] -> Value) -> [(Text, Value)] -> Value
forall a b. (a -> b) -> a -> b
$ InsOrdHashMap Text Value -> [(Text, Value)]
forall k v. InsOrdHashMap k v -> [(k, v)]
InsOrdHashMap.toList (InsOrdHashMap Text Value -> [(Text, Value)])
-> InsOrdHashMap Text Value -> [(Text, Value)]
forall a b. (a -> b) -> a -> b
$ (CompositeValue Value -> Value)
-> CompositeObject Value -> InsOrdHashMap Text Value
forall v1 v2 k.
(v1 -> v2) -> InsOrdHashMap k v1 -> InsOrdHashMap k v2
InsOrdHashMap.map CompositeValue Value -> Value
compositeValueToJSON CompositeObject Value
obj
  CVObjectArray [CompositeValue Value]
vals -> [Value] -> Value
JO.array ([Value] -> Value) -> [Value] -> Value
forall a b. (a -> b) -> a -> b
$ (CompositeValue Value -> Value)
-> [CompositeValue Value] -> [Value]
forall a b. (a -> b) -> [a] -> [b]
map CompositeValue Value -> Value
compositeValueToJSON [CompositeValue Value]
vals
  CVFromRemote Value
v -> Value
v

-- | A token used to uniquely identify the results within a join call that are
-- associated with a particular argument.
data ReplacementToken = ReplacementToken
  { -- | Unique identifier for a remote join call.
    ReplacementToken -> JoinCallId
_rtCallId :: !JoinCallId,
    -- | Unique identifier for an argument to some remote join.
    ReplacementToken -> JoinCallId
_rtArgumentId :: !JoinArgumentId
  }