2021-06-11 06:26:50 +03:00
|
|
|
module Hasura.GraphQL.Execute.RemoteJoin.Join
|
2021-09-24 01:56:37 +03:00
|
|
|
( processRemoteJoins,
|
|
|
|
)
|
|
|
|
where
|
2020-05-27 18:02:58 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
import Control.Lens ((^?))
|
|
|
|
import Data.Aeson qualified as J
|
|
|
|
import Data.Aeson.Lens (_Integral, _Number)
|
|
|
|
import Data.Aeson.Ordered qualified as JO
|
|
|
|
import Data.Environment qualified as Env
|
|
|
|
import Data.HashMap.Strict qualified as Map
|
|
|
|
import Data.HashMap.Strict.Extended qualified as Map
|
|
|
|
import Data.HashMap.Strict.InsOrd qualified as OMap
|
|
|
|
import Data.HashSet qualified as HS
|
|
|
|
import Data.IntMap.Strict qualified as IntMap
|
|
|
|
import Data.List.NonEmpty qualified as NE
|
|
|
|
import Data.Text qualified as T
|
|
|
|
import Data.Tuple (swap)
|
|
|
|
import Hasura.Base.Error
|
|
|
|
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.Types
|
|
|
|
import Hasura.GraphQL.Logging (MonadQueryLog)
|
|
|
|
import Hasura.GraphQL.Transport.Backend qualified as TB
|
|
|
|
import Hasura.GraphQL.Transport.HTTP.Protocol (GQLReqUnparsed)
|
|
|
|
import Hasura.GraphQL.Transport.Instances ()
|
|
|
|
import Hasura.Logging qualified as L
|
|
|
|
import Hasura.Prelude
|
|
|
|
import Hasura.RQL.Types
|
|
|
|
import Hasura.SQL.AnyBackend qualified as AB
|
|
|
|
import Hasura.Server.Types (RequestId)
|
|
|
|
import Hasura.Server.Version (HasVersion)
|
|
|
|
import Hasura.Session
|
|
|
|
import Hasura.Tracing qualified as Tracing
|
|
|
|
import Language.GraphQL.Draft.Syntax qualified as G
|
|
|
|
import Network.HTTP.Client qualified as HTTP
|
|
|
|
import Network.HTTP.Types qualified as N
|
2020-05-27 18:02:58 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
forRemoteJoins ::
|
|
|
|
(Applicative f) =>
|
|
|
|
Maybe RemoteJoins ->
|
|
|
|
a ->
|
|
|
|
(RemoteJoins -> f a) ->
|
|
|
|
f a
|
2021-08-06 16:39:00 +03:00
|
|
|
forRemoteJoins remoteJoins onNoJoins f =
|
|
|
|
maybe (pure onNoJoins) f remoteJoins
|
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
processRemoteJoins ::
|
|
|
|
( HasVersion,
|
|
|
|
MonadError QErr m,
|
|
|
|
MonadIO m,
|
|
|
|
EB.MonadQueryTags m,
|
|
|
|
MonadQueryLog m,
|
|
|
|
Tracing.MonadTrace m
|
|
|
|
) =>
|
|
|
|
RequestId ->
|
|
|
|
L.Logger L.Hasura ->
|
|
|
|
Env.Environment ->
|
|
|
|
HTTP.Manager ->
|
|
|
|
[N.Header] ->
|
|
|
|
UserInfo ->
|
|
|
|
EncJSON ->
|
|
|
|
Maybe RemoteJoins ->
|
|
|
|
GQLReqUnparsed ->
|
|
|
|
m EncJSON
|
2021-09-22 13:43:05 +03:00
|
|
|
processRemoteJoins requestId logger env manager reqHdrs userInfo lhs joinTree gqlreq = do
|
2021-08-06 16:39:00 +03:00
|
|
|
forRemoteJoins joinTree lhs $ \remoteJoins -> do
|
2021-09-22 13:43:05 +03:00
|
|
|
lhsParsed <- onLeft (JO.eitherDecode $ encJToLBS lhs) (throw500 . T.pack)
|
2021-09-24 01:56:37 +03:00
|
|
|
encJFromOrderedValue . runIdentity
|
|
|
|
<$> processRemoteJoins_
|
|
|
|
requestId
|
|
|
|
logger
|
|
|
|
env
|
|
|
|
manager
|
|
|
|
reqHdrs
|
|
|
|
userInfo
|
|
|
|
(Identity lhsParsed)
|
|
|
|
remoteJoins
|
|
|
|
gqlreq
|
2021-08-06 16:39:00 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
processRemoteJoins_ ::
|
|
|
|
( HasVersion,
|
|
|
|
MonadError QErr m,
|
|
|
|
MonadIO m,
|
|
|
|
EB.MonadQueryTags m,
|
|
|
|
MonadQueryLog m,
|
|
|
|
Tracing.MonadTrace m,
|
|
|
|
Traversable f
|
|
|
|
) =>
|
|
|
|
RequestId ->
|
|
|
|
L.Logger L.Hasura ->
|
|
|
|
Env.Environment ->
|
|
|
|
HTTP.Manager ->
|
|
|
|
[N.Header] ->
|
|
|
|
UserInfo ->
|
|
|
|
f JO.Value ->
|
|
|
|
RemoteJoins ->
|
|
|
|
GQLReqUnparsed ->
|
|
|
|
m (f JO.Value)
|
2021-09-22 13:43:05 +03:00
|
|
|
processRemoteJoins_ requestId logger env manager reqHdrs userInfo lhs joinTree gqlreq = do
|
2021-08-06 16:39:00 +03:00
|
|
|
(compositeValue, joins) <- collectJoinArguments (assignJoinIds joinTree) lhs
|
2021-09-24 01:56:37 +03:00
|
|
|
joinIndices <- fmap (IntMap.mapMaybe id) $
|
|
|
|
for joins $ \JoinArguments {..} -> do
|
|
|
|
let joinArguments = IntMap.fromList $ map swap $ Map.toList _jalArguments
|
|
|
|
case _jalJoin of
|
|
|
|
RemoteJoinRemoteSchema remoteSchemaJoin -> do
|
|
|
|
-- construct a remote call for
|
|
|
|
remoteCall <- RS.buildRemoteSchemaCall userInfo remoteSchemaJoin joinArguments
|
|
|
|
-- A remote call could be Nothing if there are no join arguments
|
|
|
|
for remoteCall $ \rsc@(RS.RemoteSchemaCall _ _ _ responsePaths) -> do
|
|
|
|
remoteResponse <-
|
|
|
|
RS.getRemoteSchemaResponse env manager reqHdrs userInfo rsc
|
|
|
|
-- extract the join values from the remote's response
|
|
|
|
RS.buildJoinIndex remoteResponse responsePaths
|
|
|
|
RemoteJoinSource sourceJoin childJoinTree -> AB.dispatchAnyBackend @TB.BackendTransport sourceJoin \(RemoteSourceJoin {..} :: RemoteSourceJoin b) -> do
|
|
|
|
let rows = flip map (IntMap.toList joinArguments) $ \(argumentId, argument) ->
|
2021-09-22 13:43:05 +03:00
|
|
|
Map.insert "__argument_id__" (J.toJSON argumentId) $
|
2021-09-24 01:56:37 +03:00
|
|
|
Map.fromList $
|
|
|
|
map (getFieldNameTxt *** JO.fromOrdered) $
|
|
|
|
Map.toList $ unJoinArgument argument
|
|
|
|
rowSchema = fmap (\(_, rhsType, rhsColumn) -> (rhsColumn, rhsType)) _rsjJoinColumns
|
2021-09-22 13:43:05 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
for (NE.nonEmpty rows) $ \nonEmptyRows -> do
|
|
|
|
stepInfo <-
|
|
|
|
EB.mkDBRemoteRelationshipPlan
|
|
|
|
userInfo
|
|
|
|
_rsjSource
|
|
|
|
_rsjSourceConfig
|
|
|
|
nonEmptyRows
|
|
|
|
rowSchema
|
|
|
|
(FieldName "__argument_id__")
|
|
|
|
(FieldName "f", _rsjRelationship)
|
2021-09-22 13:43:05 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
(_, sourceResponse) <-
|
|
|
|
TB.runDBQuery @b
|
|
|
|
requestId
|
|
|
|
gqlreq
|
|
|
|
-- NOTE: We're making an assumption that the 'FieldName' propagated
|
|
|
|
-- upwards from 'collectJoinArguments' is reasonable to use for
|
|
|
|
-- logging.
|
|
|
|
(G.unsafeMkName . getFieldNameTxt $ _jalFieldName)
|
|
|
|
userInfo
|
|
|
|
logger
|
|
|
|
_rsjSourceConfig
|
|
|
|
(EB.dbsiAction stepInfo)
|
|
|
|
(EB.dbsiPreparedQuery stepInfo)
|
2021-09-22 13:43:05 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
preRemoteJoinResults <- buildSourceDataJoinIndex sourceResponse
|
|
|
|
forRemoteJoins childJoinTree preRemoteJoinResults $ \childRemoteJoins -> do
|
|
|
|
results <-
|
|
|
|
processRemoteJoins_
|
|
|
|
requestId
|
|
|
|
logger
|
|
|
|
env
|
|
|
|
manager
|
|
|
|
reqHdrs
|
|
|
|
userInfo
|
|
|
|
(IntMap.elems preRemoteJoinResults)
|
|
|
|
childRemoteJoins
|
|
|
|
gqlreq
|
|
|
|
pure $ IntMap.fromAscList $ zip (IntMap.keys preRemoteJoinResults) results
|
2021-09-22 13:43:05 +03:00
|
|
|
|
2021-08-06 16:39:00 +03:00
|
|
|
joinResults joinIndices compositeValue
|
2020-05-27 18:02:58 +03:00
|
|
|
|
2021-09-22 13:43:05 +03:00
|
|
|
-- | Attempt to construct a 'JoinIndex' from some 'EncJSON' source response.
|
|
|
|
buildSourceDataJoinIndex :: MonadError QErr m => EncJSON -> m JoinIndex
|
|
|
|
buildSourceDataJoinIndex response = do
|
2021-09-24 01:56:37 +03:00
|
|
|
json <-
|
|
|
|
JO.eitherDecode (encJToLBS response) `onLeft` \err ->
|
|
|
|
throwInvalidJsonErr $ T.pack err
|
2021-09-22 13:43:05 +03:00
|
|
|
case json of
|
|
|
|
JO.Array arr -> fmap IntMap.fromList $ for (toList arr) \case
|
|
|
|
JO.Object obj -> do
|
|
|
|
argumentResult <-
|
|
|
|
JO.lookup "f" obj
|
|
|
|
`onNothing` throwMissingRelationshipDataErr
|
|
|
|
argumentIdValue <-
|
|
|
|
JO.lookup "__argument_id__" obj
|
|
|
|
`onNothing` throwMissingArgumentIdErr
|
|
|
|
argumentId <-
|
|
|
|
(argumentIdValue ^? _Number . _Integral)
|
|
|
|
`onNothing` throwInvalidArgumentIdTypeErr
|
|
|
|
pure (argumentId, argumentResult)
|
|
|
|
_ -> throwNoNestedObjectErr
|
|
|
|
_ -> throwNoListOfObjectsErr
|
|
|
|
where
|
|
|
|
throwInvalidJsonErr errMsg =
|
|
|
|
throw500 $
|
|
|
|
"failed to decode JSON response from the source: " <> errMsg
|
|
|
|
throwMissingRelationshipDataErr =
|
|
|
|
throw500 $
|
|
|
|
"cannot find relationship data (aliased as 'f') within the source \
|
|
|
|
\response"
|
|
|
|
throwMissingArgumentIdErr =
|
|
|
|
throw500 $
|
|
|
|
"cannot find '__argument_id__' within the source response"
|
|
|
|
throwInvalidArgumentIdTypeErr =
|
|
|
|
throw500 $ "expected 'argument_id' to be an integer"
|
|
|
|
throwNoNestedObjectErr =
|
|
|
|
throw500 $
|
|
|
|
"expected an object one level deep in the remote schema's response, \
|
|
|
|
\but found an array/scalar value instead"
|
|
|
|
throwNoListOfObjectsErr =
|
|
|
|
throw500 $
|
|
|
|
"expected a list of objects in the remote schema's response, but found \
|
|
|
|
\an object/scalar value instead"
|
|
|
|
|
2020-05-27 18:02:58 +03:00
|
|
|
type CompositeObject a = OMap.InsOrdHashMap Text (CompositeValue a)
|
|
|
|
|
|
|
|
-- | A hybrid JSON value representation which captures the context of remote join field in type parameter.
|
|
|
|
data CompositeValue a
|
2021-09-22 13:43:05 +03:00
|
|
|
= CVOrdValue !JO.Value
|
2020-05-27 18:02:58 +03:00
|
|
|
| CVObject !(CompositeObject a)
|
|
|
|
| CVObjectArray ![CompositeValue a]
|
|
|
|
| CVFromRemote !a
|
|
|
|
deriving (Show, Eq, Functor, Foldable, Traversable)
|
|
|
|
|
2021-09-22 13:43:05 +03:00
|
|
|
compositeValueToJSON :: CompositeValue JO.Value -> JO.Value
|
2020-05-27 18:02:58 +03:00
|
|
|
compositeValueToJSON = \case
|
2021-09-24 01:56:37 +03:00
|
|
|
CVOrdValue v -> v
|
|
|
|
CVObject obj -> JO.object $ OMap.toList $ OMap.map compositeValueToJSON obj
|
2021-09-22 13:43:05 +03:00
|
|
|
CVObjectArray vals -> JO.array $ map compositeValueToJSON vals
|
2021-09-24 01:56:37 +03:00
|
|
|
CVFromRemote v -> v
|
2020-05-27 18:02:58 +03:00
|
|
|
|
2021-08-06 16:39:00 +03:00
|
|
|
-- | A token used to uniquely identify the results within a join call that are
|
|
|
|
-- associated with a particular argument.
|
2021-09-24 01:56:37 +03:00
|
|
|
data ReplacementToken = ReplacementToken
|
|
|
|
{ -- | Unique identifier for a remote join call.
|
|
|
|
_rtCallId :: !JoinCallId,
|
|
|
|
-- | Unique identifier for an argument to some remote join.
|
|
|
|
_rtArgumentId :: !JoinArgumentId
|
|
|
|
}
|
2021-08-06 16:39:00 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
joinResults ::
|
|
|
|
forall f m.
|
|
|
|
(MonadError QErr m, Traversable f) =>
|
|
|
|
IntMap.IntMap (IntMap.IntMap JO.Value) ->
|
|
|
|
f (CompositeValue ReplacementToken) ->
|
|
|
|
m (f JO.Value)
|
2021-08-06 16:39:00 +03:00
|
|
|
joinResults remoteResults compositeValues = do
|
|
|
|
traverse (fmap compositeValueToJSON . traverse replaceToken) compositeValues
|
2020-05-27 18:02:58 +03:00
|
|
|
where
|
2021-09-22 13:43:05 +03:00
|
|
|
replaceToken :: ReplacementToken -> m JO.Value
|
2021-08-06 16:39:00 +03:00
|
|
|
replaceToken (ReplacementToken joinCallId argumentId) = do
|
2021-09-24 01:56:37 +03:00
|
|
|
joinCallResults <-
|
|
|
|
onNothing (IntMap.lookup joinCallId remoteResults) $
|
|
|
|
throw500 $
|
|
|
|
"couldn't find results for the join with id: "
|
|
|
|
<> tshow joinCallId
|
2021-08-06 16:39:00 +03:00
|
|
|
onNothing (IntMap.lookup argumentId joinCallResults) $
|
2021-09-24 01:56:37 +03:00
|
|
|
throw500 $
|
|
|
|
"couldn't find a value for argument id in the join results: "
|
|
|
|
<> tshow (argumentId, joinCallId)
|
2021-08-06 16:39:00 +03:00
|
|
|
|
|
|
|
-- | 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).
|
|
|
|
assignJoinIds :: JoinTree RemoteJoin -> JoinTree (JoinCallId, RemoteJoin)
|
|
|
|
assignJoinIds joinTree =
|
|
|
|
evalState (traverse assignId joinTree) (0, [])
|
2020-05-27 18:02:58 +03:00
|
|
|
where
|
2021-09-24 01:56:37 +03:00
|
|
|
assignId ::
|
|
|
|
RemoteJoin ->
|
|
|
|
State (JoinCallId, [(JoinCallId, RemoteJoin)]) (JoinCallId, RemoteJoin)
|
2021-08-06 16:39:00 +03:00
|
|
|
assignId remoteJoin = do
|
|
|
|
(joinCallId, joinIds) <- get
|
2021-09-22 13:43:05 +03:00
|
|
|
let mJoinId = joinIds & find \(_, j) -> j == remoteJoin
|
2021-08-06 16:39:00 +03:00
|
|
|
mJoinId `onNothing` do
|
2021-09-24 01:56:37 +03:00
|
|
|
put (joinCallId + 1, (joinCallId, remoteJoin) : joinIds)
|
2021-08-06 16:39:00 +03:00
|
|
|
pure (joinCallId, remoteJoin)
|
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
collectJoinArguments ::
|
|
|
|
forall f m.
|
|
|
|
(MonadError QErr m, Traversable f) =>
|
|
|
|
JoinTree (JoinCallId, RemoteJoin) ->
|
|
|
|
f JO.Value ->
|
|
|
|
m (f (CompositeValue ReplacementToken), IntMap.IntMap JoinArguments)
|
2021-08-06 16:39:00 +03:00
|
|
|
collectJoinArguments joinTree lhs = do
|
|
|
|
result <- flip runStateT (0, mempty) $ traverse (traverseValue joinTree) lhs
|
|
|
|
-- Discard the 'JoinArgumentId' from the intermediate state transformation.
|
|
|
|
pure $ second snd result
|
2020-05-27 18:02:58 +03:00
|
|
|
where
|
2021-09-24 01:56:37 +03:00
|
|
|
getReplacementToken ::
|
|
|
|
IntMap.Key ->
|
|
|
|
RemoteJoin ->
|
|
|
|
JoinArgument ->
|
|
|
|
FieldName ->
|
|
|
|
StateT
|
|
|
|
(JoinArgumentId, IntMap.IntMap JoinArguments)
|
|
|
|
m
|
|
|
|
ReplacementToken
|
2021-09-22 13:43:05 +03:00
|
|
|
getReplacementToken joinId remoteJoin argument fieldName = do
|
2021-08-06 16:39:00 +03:00
|
|
|
(counter, joins) <- get
|
|
|
|
case IntMap.lookup joinId joins of
|
2021-09-22 13:43:05 +03:00
|
|
|
-- 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 arguments _fieldName) ->
|
2021-08-06 16:39:00 +03:00
|
|
|
case Map.lookup argument arguments of
|
|
|
|
Just argumentId -> pure $ ReplacementToken joinId argumentId
|
2021-09-24 01:56:37 +03:00
|
|
|
Nothing -> addNewArgument counter joins arguments
|
2021-08-06 16:39:00 +03:00
|
|
|
Nothing -> addNewArgument counter joins mempty
|
|
|
|
where
|
|
|
|
addNewArgument counter joins arguments = do
|
|
|
|
let argumentId = counter
|
2021-09-22 13:43:05 +03:00
|
|
|
newArguments =
|
|
|
|
JoinArguments
|
|
|
|
remoteJoin
|
|
|
|
(Map.insert argument argumentId arguments)
|
|
|
|
fieldName
|
2021-08-06 16:39:00 +03:00
|
|
|
put (counter + 1, IntMap.insert joinId newArguments joins)
|
|
|
|
pure $ ReplacementToken joinId argumentId
|
2020-05-27 18:02:58 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
traverseValue ::
|
|
|
|
JoinTree (IntMap.Key, RemoteJoin) ->
|
|
|
|
JO.Value ->
|
|
|
|
StateT
|
|
|
|
(JoinArgumentId, IntMap.IntMap JoinArguments)
|
|
|
|
m
|
|
|
|
(CompositeValue ReplacementToken)
|
2021-08-06 16:39:00 +03:00
|
|
|
traverseValue joinTree_ = \case
|
2021-09-22 13:43:05 +03:00
|
|
|
-- 'JO.Null' is a special case of scalar value here, which indicates that
|
2021-08-06 16:39:00 +03:00
|
|
|
-- 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 {
|
|
|
|
-- }
|
|
|
|
-- }
|
|
|
|
-- }
|
|
|
|
-- }
|
|
|
|
-- }
|
2021-09-24 01:56:37 +03:00
|
|
|
JO.Null -> pure $ CVOrdValue JO.Null
|
2021-09-22 13:43:05 +03:00
|
|
|
JO.Object object -> CVObject <$> traverseObject joinTree_ object
|
2021-09-24 01:56:37 +03:00
|
|
|
JO.Array array -> CVObjectArray <$> mapM (traverseValue joinTree_) (toList array)
|
|
|
|
_ -> throw500 "found a scalar value when traversing with a non-empty join tree"
|
2021-08-06 16:39:00 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
traverseObject ::
|
|
|
|
JoinTree (IntMap.Key, RemoteJoin) ->
|
|
|
|
JO.Object ->
|
|
|
|
StateT
|
|
|
|
(JoinArgumentId, IntMap.IntMap JoinArguments)
|
|
|
|
m
|
|
|
|
(InsOrdHashMap Text (CompositeValue ReplacementToken))
|
2021-08-06 16:39:00 +03:00
|
|
|
traverseObject joinTree_ object = do
|
2021-09-24 01:56:37 +03:00
|
|
|
let phantomFields =
|
|
|
|
HS.fromList $
|
|
|
|
map getFieldNameTxt $
|
|
|
|
concatMap (getPhantomFields . snd) $ toList joinTree_
|
2021-08-06 16:39:00 +03:00
|
|
|
|
2021-09-24 01:56:37 +03:00
|
|
|
joinTreeNodes =
|
|
|
|
Map.mapKeys getFieldNameTxt $
|
|
|
|
Map.fromList $
|
|
|
|
NE.toList $ unJoinTree joinTree_
|
2021-08-06 16:39:00 +03:00
|
|
|
|
|
|
|
-- 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
|
2021-09-22 13:43:05 +03:00
|
|
|
compositeObject <- for (JO.toList object) $ \(fieldName, value_) ->
|
2021-08-06 16:39:00 +03:00
|
|
|
(fieldName,) <$> case Map.lookup fieldName joinTreeNodes of
|
|
|
|
Just (Leaf (joinId, remoteJoin)) -> do
|
|
|
|
joinArgument <- forM (getJoinColumnMapping remoteJoin) $ \alias -> do
|
|
|
|
let aliasTxt = getFieldNameTxt $ getAliasFieldName alias
|
2021-09-22 13:43:05 +03:00
|
|
|
onNothing (JO.lookup aliasTxt object) $
|
2021-08-06 16:39:00 +03:00
|
|
|
throw500 $ "a join column is missing from the response: " <> aliasTxt
|
2021-09-22 13:43:05 +03:00
|
|
|
if Map.null (Map.filter (== JO.Null) joinArgument)
|
2021-09-24 01:56:37 +03:00
|
|
|
then
|
|
|
|
Just . CVFromRemote
|
|
|
|
<$> getReplacementToken joinId remoteJoin (JoinArgument joinArgument) (FieldName fieldName)
|
|
|
|
else -- we do not join with the remote field if any of the leaves of
|
|
|
|
-- the join argument are null
|
|
|
|
pure $ Just $ CVOrdValue JO.Null
|
2021-08-06 16:39:00 +03:00
|
|
|
Just (Tree joinSubTree) ->
|
|
|
|
Just <$> traverseValue joinSubTree value_
|
|
|
|
Nothing ->
|
|
|
|
if HS.member fieldName phantomFields
|
2021-09-24 01:56:37 +03:00
|
|
|
then pure Nothing
|
|
|
|
else pure $ Just $ CVOrdValue value_
|
2021-08-06 16:39:00 +03:00
|
|
|
|
|
|
|
pure . OMap.fromList $
|
|
|
|
-- filter out the Nothings
|
|
|
|
mapMaybe sequenceA compositeObject
|