graphql-engine/server/src-lib/Hasura/Backends/DataConnector/Plan/QueryPlan.hs
David Overton 346804fc67 Support nested object fields in DC API and use this to implement nest…
## Description

This change adds support for nested object fields in HGE IR and Schema Cache, the Data Connectors backend and API, and the MongoDB agent.

### Data Connector API changes

- The `/schema` endpoint response now includes an optional set of GraphQL type definitions. Table column types can refer to these definitions by name.
- Queries can now include a new field type `object` which contains a column name and a nested query. This allows querying into a nested object within a field.

### MongoDB agent changes

- Add support for querying into nested documents using the new `object` field type.

### HGE changes

- The `Backend` type class has a new type family `XNestedObjects b` which controls whether or not a backend supports querying into nested objects. This is currently enabled only for the `DataConnector` backend.
- For backends that support nested objects, the `FieldInfo` type gets a new constructor `FINestedObject`, and the `AnnFieldG` type gets a new constructor `AFNestedObject`.
- If the DC `/schema` endpoint returns any custom GraphQL type definitions they are stored in the `TableInfo` for each table in the source.
- During schema cache building, the function `addNonColumnFields` will check whether any column types match custom GraphQL object types stored in the `TableInfo`. If so, they are converted into `FINestedObject` instead of `FIColumn` in the `FieldInfoMap`.
- When building the `FieldParser`s from `FieldInfo` (function `fieldSelection`) any `FINestedObject` fields are converted into nested object parsers returning `AFNestedObject`.
- The `DataConnector` query planner converts `AFNestedObject` fields into `object` field types in the query sent to the agent.

## Limitations

### HGE not yet implemented:
- Support for nested arrays
- Support for nested objects/arrays in mutations
- Support for nested objects/arrays in order-by
- Support for filters (`where`) in nested objects/arrays
- Support for adding custom GraphQL types via track table metadata API
- Support for interface and union types
- Tests for nested objects

### Mongo agent not yet implemented:

- Generate nested object types from validation schema
- Support for aggregates
- Support for order-by
- Configure agent port
- Build agent in CI
- Agent tests for nested objects and MongoDB agent

PR-URL: https://github.com/hasura/graphql-engine-mono/pull/7844
GitOrigin-RevId: aec9ec1e4216293286a68f9b1af6f3f5317db423
2023-04-11 01:30:37 +00:00

611 lines
28 KiB
Haskell

module Hasura.Backends.DataConnector.Plan.QueryPlan
( -- Main external interface
mkQueryPlan,
queryHasRelations,
-- Internals reused by other plan modules
translateAnnSimpleSelectToQueryRequest,
translateAnnAggregateSelectToQueryRequest,
translateAnnFields,
reshapeSimpleSelectRows,
reshapeTableAggregateFields,
reshapeAnnFields,
)
where
--------------------------------------------------------------------------------
import Control.Monad.Trans.Writer.CPS qualified as CPS
import Data.Aeson qualified as J
import Data.Aeson.Encoding qualified as JE
import Data.Bifunctor (Bifunctor (bimap))
import Data.Has (Has)
import Data.HashMap.Strict qualified as HashMap
import Data.List.NonEmpty qualified as NE
import Data.Semigroup (Min (..))
import Hasura.Backends.DataConnector.API qualified as API
import Hasura.Backends.DataConnector.Adapter.Backend
import Hasura.Backends.DataConnector.Adapter.Types
import Hasura.Backends.DataConnector.Plan.Common
import Hasura.Base.Error
import Hasura.Prelude
import Hasura.RQL.IR.BoolExp
import Hasura.RQL.IR.OrderBy
import Hasura.RQL.IR.Select
import Hasura.RQL.IR.Value
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.SQL.Backend
import Hasura.Session
import Language.GraphQL.Draft.Syntax qualified as G
import Witch qualified
--------------------------------------------------------------------------------
data FieldsAndAggregates = FieldsAndAggregates
{ _faaFields :: Maybe (HashMap FieldName API.Field),
_faaAggregates :: Maybe (HashMap FieldName API.Aggregate)
}
deriving stock (Show, Eq)
instance Semigroup FieldsAndAggregates where
left <> right =
FieldsAndAggregates
(_faaFields left <> _faaFields right)
(_faaAggregates left <> _faaAggregates right)
instance Monoid FieldsAndAggregates where
mempty = FieldsAndAggregates Nothing Nothing
--------------------------------------------------------------------------------
-- | Map a 'QueryDB 'DataConnector' term into a 'Plan'
mkQueryPlan ::
forall m.
MonadError QErr m =>
SessionVariables ->
SourceConfig ->
QueryDB 'DataConnector Void (UnpreparedValue 'DataConnector) ->
m (Plan API.QueryRequest API.QueryResponse)
mkQueryPlan sessionVariables (SourceConfig {}) ir = do
queryRequest <- translateQueryDB ir
pure $ Plan queryRequest (reshapeResponseToQueryShape ir)
where
translateQueryDB ::
QueryDB 'DataConnector Void (UnpreparedValue 'DataConnector) ->
m API.QueryRequest
translateQueryDB =
\case
QDBMultipleRows simpleSelect -> translateAnnSimpleSelectToQueryRequest sessionVariables simpleSelect
QDBSingleRow simpleSelect -> translateAnnSimpleSelectToQueryRequest sessionVariables simpleSelect
QDBAggregation aggregateSelect -> translateAnnAggregateSelectToQueryRequest sessionVariables aggregateSelect
translateAnnSimpleSelectToQueryRequest ::
forall m.
MonadError QErr m =>
SessionVariables ->
AnnSimpleSelectG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
m API.QueryRequest
translateAnnSimpleSelectToQueryRequest sessionVariables simpleSelect =
translateAnnSelectToQueryRequest sessionVariables (translateAnnFieldsWithNoAggregates sessionVariables noPrefix) simpleSelect
translateAnnAggregateSelectToQueryRequest ::
forall m.
MonadError QErr m =>
SessionVariables ->
AnnAggregateSelectG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
m API.QueryRequest
translateAnnAggregateSelectToQueryRequest sessionVariables aggregateSelect =
translateAnnSelectToQueryRequest sessionVariables (translateTableAggregateFields sessionVariables) aggregateSelect
translateAnnSelectToQueryRequest ::
forall m fieldType.
MonadError QErr m =>
SessionVariables ->
(API.TableName -> Fields (fieldType (UnpreparedValue 'DataConnector)) -> CPS.WriterT TableRelationships m FieldsAndAggregates) ->
AnnSelectG 'DataConnector fieldType (UnpreparedValue 'DataConnector) ->
m API.QueryRequest
translateAnnSelectToQueryRequest sessionVariables translateFieldsAndAggregates selectG = do
tableName <- extractTableName selectG
(query, (TableRelationships tableRelationships)) <- CPS.runWriterT (translateAnnSelect sessionVariables translateFieldsAndAggregates tableName selectG)
let apiTableRelationships = uncurry API.TableRelationships <$> HashMap.toList tableRelationships
pure $
API.QueryRequest
{ _qrTable = tableName,
_qrTableRelationships = apiTableRelationships,
_qrQuery = query,
_qrForeach = Nothing
}
extractTableName :: MonadError QErr m => AnnSelectG 'DataConnector fieldsType valueType -> m API.TableName
extractTableName selectG =
case _asnFrom selectG of
FromTable tn -> pure $ Witch.from tn
FromIdentifier _ -> throw400 NotSupported "AnnSelectG: FromIdentifier not supported"
FromFunction {} -> throw400 NotSupported "AnnSelectG: FromFunction not supported"
FromLogicalModel {} -> throw400 NotSupported "AnnSelectG: FromLogicalModel not supported"
translateAnnSelect ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
(API.TableName -> Fields (fieldType (UnpreparedValue 'DataConnector)) -> CPS.WriterT writerOutput m FieldsAndAggregates) ->
API.TableName ->
AnnSelectG 'DataConnector fieldType (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m API.Query
translateAnnSelect sessionVariables translateFieldsAndAggregates tableName selectG = do
FieldsAndAggregates {..} <- translateFieldsAndAggregates tableName (_asnFields selectG)
let whereClauseWithPermissions =
case _saWhere (_asnArgs selectG) of
Just expr -> BoolAnd [expr, _tpFilter (_asnPerm selectG)]
Nothing -> _tpFilter (_asnPerm selectG)
whereClause <- translateBoolExpToExpression sessionVariables tableName whereClauseWithPermissions
orderBy <- traverse (translateOrderBy sessionVariables tableName) (_saOrderBy $ _asnArgs selectG)
pure
API.Query
{ _qFields = mapFieldNameHashMap <$> _faaFields,
_qAggregates = mapFieldNameHashMap <$> _faaAggregates,
_qAggregatesLimit = _saLimit (_asnArgs selectG) <* _faaAggregates, -- Only include the aggregates limit if we actually have aggregrates
_qLimit =
fmap getMin $
foldMap
(fmap Min)
[ _saLimit (_asnArgs selectG),
_tpLimit (_asnPerm selectG)
],
_qOffset = fmap fromIntegral (_saOffset (_asnArgs selectG)),
_qWhere = whereClause,
_qOrderBy = orderBy
}
translateOrderBy ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
NE.NonEmpty (AnnotatedOrderByItemG 'DataConnector (UnpreparedValue 'DataConnector)) ->
CPS.WriterT writerOutput m API.OrderBy
translateOrderBy sessionVariables sourceTableName orderByItems = do
orderByElementsAndRelations <- for orderByItems \OrderByItemG {..} -> do
let orderDirection = maybe API.Ascending Witch.from obiType
translateOrderByElement sessionVariables sourceTableName orderDirection [] obiColumn
relations <- lift . mergeOrderByRelations $ snd <$> orderByElementsAndRelations
pure
API.OrderBy
{ _obRelations = relations,
_obElements = fst <$> orderByElementsAndRelations
}
translateOrderByElement ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
API.OrderDirection ->
[API.RelationshipName] ->
AnnotatedOrderByElement 'DataConnector (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m (API.OrderByElement, HashMap API.RelationshipName API.OrderByRelation)
translateOrderByElement sessionVariables sourceTableName orderDirection targetReversePath = \case
AOCColumn (ColumnInfo {..}) ->
pure
( API.OrderByElement
{ _obeTargetPath = reverse targetReversePath,
_obeTarget = API.OrderByColumn $ Witch.from ciColumn,
_obeOrderDirection = orderDirection
},
mempty
)
AOCObjectRelation relationshipInfo filterExp orderByElement -> do
(relationshipName, API.Relationship {..}) <- recordTableRelationshipFromRelInfo sourceTableName relationshipInfo
(translatedOrderByElement, subOrderByRelations) <- translateOrderByElement sessionVariables _rTargetTable orderDirection (relationshipName : targetReversePath) orderByElement
targetTableWhereExp <- translateBoolExpToExpression sessionVariables _rTargetTable filterExp
let orderByRelations = HashMap.fromList [(relationshipName, API.OrderByRelation targetTableWhereExp subOrderByRelations)]
pure (translatedOrderByElement, orderByRelations)
AOCArrayAggregation relationshipInfo filterExp aggregateOrderByElement -> do
(relationshipName, API.Relationship {..}) <- recordTableRelationshipFromRelInfo sourceTableName relationshipInfo
orderByTarget <- case aggregateOrderByElement of
AAOCount ->
pure API.OrderByStarCountAggregate
AAOOp aggFunctionTxt resultType ColumnInfo {..} -> do
aggFunction <- lift $ translateSingleColumnAggregateFunction aggFunctionTxt
let resultScalarType = Witch.from $ columnTypeToScalarType resultType
pure . API.OrderBySingleColumnAggregate $ API.SingleColumnAggregate aggFunction (Witch.from ciColumn) resultScalarType
let translatedOrderByElement =
API.OrderByElement
{ _obeTargetPath = reverse (relationshipName : targetReversePath),
_obeTarget = orderByTarget,
_obeOrderDirection = orderDirection
}
targetTableWhereExp <- translateBoolExpToExpression sessionVariables _rTargetTable filterExp
let orderByRelations = HashMap.fromList [(relationshipName, API.OrderByRelation targetTableWhereExp mempty)]
pure (translatedOrderByElement, orderByRelations)
mergeOrderByRelations ::
forall m f.
(MonadError QErr m, Foldable f) =>
f (HashMap API.RelationshipName API.OrderByRelation) ->
m (HashMap API.RelationshipName API.OrderByRelation)
mergeOrderByRelations orderByRelationsList =
foldM mergeMap mempty orderByRelationsList
where
mergeMap :: HashMap API.RelationshipName API.OrderByRelation -> HashMap API.RelationshipName API.OrderByRelation -> m (HashMap API.RelationshipName API.OrderByRelation)
mergeMap left right = foldM (\targetMap (relName, orderByRel) -> HashMap.alterF (maybe (pure $ Just orderByRel) (fmap Just . mergeOrderByRelation orderByRel)) relName targetMap) left $ HashMap.toList right
mergeOrderByRelation :: API.OrderByRelation -> API.OrderByRelation -> m API.OrderByRelation
mergeOrderByRelation right left =
if API._obrWhere left == API._obrWhere right
then do
mergedSubrelations <- mergeMap (API._obrSubrelations left) (API._obrSubrelations right)
pure $ API.OrderByRelation (API._obrWhere left) mergedSubrelations
else throw500 "mergeOrderByRelations: Differing filter expressions found for the same table"
translateAnnFieldsWithNoAggregates ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
FieldPrefix ->
API.TableName ->
AnnFieldsG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m FieldsAndAggregates
translateAnnFieldsWithNoAggregates sessionVariables fieldNamePrefix sourceTableName fields =
(\fields' -> FieldsAndAggregates (Just fields') Nothing) <$> translateAnnFields sessionVariables fieldNamePrefix sourceTableName fields
translateAnnFields ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
FieldPrefix ->
API.TableName ->
AnnFieldsG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m (HashMap FieldName API.Field)
translateAnnFields sessionVariables fieldNamePrefix sourceTableName fields = do
translatedFields <- traverse (traverse (translateAnnField sessionVariables sourceTableName)) fields
pure $ HashMap.fromList (mapMaybe (\(fieldName, field) -> (applyPrefix fieldNamePrefix fieldName,) <$> field) translatedFields)
translateAnnField ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
AnnFieldG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m (Maybe API.Field)
translateAnnField sessionVariables sourceTableName = \case
AFColumn colField ->
-- TODO: make sure certain fields in colField are not in use, since we don't
-- support them
pure . Just $ API.ColumnField (Witch.from $ _acfColumn colField) (Witch.from . columnTypeToScalarType $ _acfType colField)
AFObjectRelation objRel -> do
let targetTable = Witch.from $ _aosTableFrom (_aarAnnSelect objRel)
let relationshipName = mkRelationshipName $ _aarRelationshipName objRel
fields <- translateAnnFields sessionVariables noPrefix targetTable (_aosFields (_aarAnnSelect objRel))
whereClause <- translateBoolExpToExpression sessionVariables targetTable (_aosTableFilter (_aarAnnSelect objRel))
recordTableRelationship
sourceTableName
relationshipName
API.Relationship
{ _rTargetTable = targetTable,
_rRelationshipType = API.ObjectRelationship,
_rColumnMapping = HashMap.fromList $ bimap Witch.from Witch.from <$> HashMap.toList (_aarColumnMapping objRel)
}
pure . Just . API.RelField $
API.RelationshipField
relationshipName
( API.Query
{ _qFields = Just $ mapFieldNameHashMap fields,
_qAggregates = mempty,
_qWhere = whereClause,
_qAggregatesLimit = Nothing,
_qLimit = Nothing,
_qOffset = Nothing,
_qOrderBy = Nothing
}
)
AFArrayRelation (ASSimple arrayRelationSelect) -> do
Just <$> translateArrayRelationSelect sessionVariables sourceTableName (translateAnnFieldsWithNoAggregates sessionVariables noPrefix) arrayRelationSelect
AFArrayRelation (ASAggregate arrayRelationSelect) ->
Just <$> translateArrayRelationSelect sessionVariables sourceTableName (translateTableAggregateFields sessionVariables) arrayRelationSelect
AFExpression _literal ->
-- We ignore literal text fields (we don't send them to the data connector agent)
-- and add them back to the response JSON when we reshape what the agent returns
-- to us
pure Nothing
AFNestedObject nestedObj ->
Just . API.NestedObjField (Witch.from $ _anosColumn nestedObj)
<$> translateNestedObjectSelect sessionVariables sourceTableName nestedObj
translateArrayRelationSelect ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
(API.TableName -> Fields (fieldType (UnpreparedValue 'DataConnector)) -> CPS.WriterT writerOutput m FieldsAndAggregates) ->
AnnRelationSelectG 'DataConnector (AnnSelectG 'DataConnector fieldType (UnpreparedValue 'DataConnector)) ->
CPS.WriterT writerOutput m API.Field
translateArrayRelationSelect sessionVariables sourceTableName translateFieldsAndAggregates arrRel = do
targetTable <- lift $ extractTableName (_aarAnnSelect arrRel)
query <- translateAnnSelect sessionVariables translateFieldsAndAggregates targetTable (_aarAnnSelect arrRel)
let relationshipName = mkRelationshipName $ _aarRelationshipName arrRel
recordTableRelationship
sourceTableName
relationshipName
API.Relationship
{ _rTargetTable = targetTable,
_rRelationshipType = API.ArrayRelationship,
_rColumnMapping = HashMap.fromList $ bimap Witch.from Witch.from <$> HashMap.toList (_aarColumnMapping arrRel)
}
pure . API.RelField $
API.RelationshipField
relationshipName
query
translateTableAggregateFields ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
TableAggregateFieldsG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m FieldsAndAggregates
translateTableAggregateFields sessionVariables sourceTableName fields = do
mconcat <$> traverse (uncurry (translateTableAggregateField sessionVariables sourceTableName)) fields
translateTableAggregateField ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
FieldName ->
TableAggregateFieldG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m FieldsAndAggregates
translateTableAggregateField sessionVariables sourceTableName fieldName = \case
TAFAgg aggregateFields -> do
let fieldNamePrefix = prefixWith fieldName
translatedAggregateFields <- lift $ mconcat <$> traverse (uncurry (translateAggregateField fieldNamePrefix)) aggregateFields
pure $
FieldsAndAggregates
Nothing
(Just translatedAggregateFields)
TAFNodes _ fields ->
translateAnnFieldsWithNoAggregates sessionVariables (prefixWith fieldName) sourceTableName fields
TAFExp _txt ->
-- We ignore literal text fields (we don't send them to the data connector agent)
-- and add them back to the response JSON when we reshape what the agent returns
-- to us
pure mempty
translateAggregateField ::
MonadError QErr m =>
FieldPrefix ->
FieldName ->
AggregateField 'DataConnector ->
m (HashMap FieldName API.Aggregate)
translateAggregateField fieldPrefix fieldName = \case
AFCount countAggregate ->
let aggregate =
case countAggregate of
StarCount -> API.StarCount
ColumnCount column -> API.ColumnCount $ API.ColumnCountAggregate {_ccaColumn = Witch.from column, _ccaDistinct = False}
ColumnDistinctCount column -> API.ColumnCount $ API.ColumnCountAggregate {_ccaColumn = Witch.from column, _ccaDistinct = True}
in pure $ HashMap.singleton (applyPrefix fieldPrefix fieldName) aggregate
AFOp AggregateOp {..} -> do
let fieldPrefix' = fieldPrefix <> prefixWith fieldName
aggFunction <- translateSingleColumnAggregateFunction _aoOp
fmap (HashMap.fromList . catMaybes) . forM _aoFields $ \(columnFieldName, columnField) ->
case columnField of
CFCol column resultType ->
let resultScalarType = Witch.from $ columnTypeToScalarType resultType
in pure . Just $ (applyPrefix fieldPrefix' columnFieldName, API.SingleColumn $ API.SingleColumnAggregate aggFunction (Witch.from column) resultScalarType)
CFExp _txt ->
-- We ignore literal text fields (we don't send them to the data connector agent)
-- and add them back to the response JSON when we reshape what the agent returns
-- to us
pure Nothing
AFExp _txt ->
-- We ignore literal text fields (we don't send them to the data connector agent)
-- and add them back to the response JSON when we reshape what the agent returns
-- to us
pure mempty
translateSingleColumnAggregateFunction :: MonadError QErr m => Text -> m API.SingleColumnAggregateFunction
translateSingleColumnAggregateFunction functionName =
fmap API.SingleColumnAggregateFunction (G.mkName functionName)
`onNothing` throw500 ("translateSingleColumnAggregateFunction: Invalid aggregate function encountered: " <> functionName)
translateNestedObjectSelect ::
( Has TableRelationships writerOutput,
Monoid writerOutput,
MonadError QErr m
) =>
SessionVariables ->
API.TableName ->
AnnNestedObjectSelectG 'DataConnector Void (UnpreparedValue 'DataConnector) ->
CPS.WriterT writerOutput m API.Query
translateNestedObjectSelect sessionVariables tableName selectG = do
FieldsAndAggregates {..} <- translateAnnFieldsWithNoAggregates sessionVariables noPrefix tableName $ _anosFields selectG
pure
API.Query
{ _qFields = mapFieldNameHashMap <$> _faaFields,
_qAggregates = Nothing,
_qAggregatesLimit = Nothing,
_qLimit = Nothing,
_qOffset = Nothing,
_qWhere = Nothing,
_qOrderBy = Nothing
}
--------------------------------------------------------------------------------
-- | Validate if a 'API.QueryRequest' contains any relationships.
queryHasRelations :: API.QueryRequest -> Bool
queryHasRelations API.QueryRequest {..} = _qrTableRelationships /= mempty
--------------------------------------------------------------------------------
reshapeResponseToQueryShape ::
MonadError QErr m =>
QueryDB 'DataConnector Void v ->
API.QueryResponse ->
m J.Encoding
reshapeResponseToQueryShape queryDb response =
case queryDb of
QDBMultipleRows simpleSelect -> reshapeSimpleSelectRows Many (_asnFields simpleSelect) response
QDBSingleRow simpleSelect -> reshapeSimpleSelectRows Single (_asnFields simpleSelect) response
QDBAggregation aggregateSelect -> reshapeTableAggregateFields (_asnFields aggregateSelect) response
reshapeSimpleSelectRows ::
MonadError QErr m =>
Cardinality ->
AnnFieldsG 'DataConnector Void v ->
API.QueryResponse ->
m J.Encoding
reshapeSimpleSelectRows cardinality fields API.QueryResponse {..} =
case cardinality of
Single ->
case rows of
[] -> pure $ J.toEncoding J.Null
[singleRow] -> reshapeAnnFields noPrefix fields singleRow
_multipleRows ->
throw500 "Data Connector agent returned multiple rows when only one was expected" -- TODO(dchambers): Add pathing information for error clarity
Many -> do
reshapedRows <- traverse (reshapeAnnFields noPrefix fields) rows
pure $ JE.list id reshapedRows
where
rows = fromMaybe mempty _qrRows
reshapeTableAggregateFields ::
MonadError QErr m =>
TableAggregateFieldsG 'DataConnector Void v ->
API.QueryResponse ->
m J.Encoding
reshapeTableAggregateFields tableAggregateFields API.QueryResponse {..} = do
reshapedFields <- forM tableAggregateFields $ \(fieldName@(FieldName fieldNameText), tableAggregateField) -> do
case tableAggregateField of
TAFAgg aggregateFields -> do
reshapedAggregateFields <- reshapeAggregateFields (prefixWith fieldName) aggregateFields responseAggregates
pure $ (fieldNameText, reshapedAggregateFields)
TAFNodes _ annFields -> do
reshapedRows <- traverse (reshapeAnnFields (prefixWith fieldName) annFields) responseRows
pure $ (fieldNameText, JE.list id reshapedRows)
TAFExp txt ->
pure $ (fieldNameText, JE.text txt)
pure $ encodeAssocListAsObject reshapedFields
where
responseRows = fromMaybe mempty _qrRows
responseAggregates = fromMaybe mempty _qrAggregates
reshapeAggregateFields ::
MonadError QErr m =>
FieldPrefix ->
AggregateFields 'DataConnector ->
HashMap API.FieldName J.Value ->
m J.Encoding
reshapeAggregateFields fieldPrefix aggregateFields responseAggregates = do
reshapedFields <- forM aggregateFields $ \(fieldName@(FieldName fieldNameText), aggregateField) ->
case aggregateField of
AFCount _countAggregate -> do
let fieldNameKey = API.FieldName . getFieldNameTxt $ applyPrefix fieldPrefix fieldName
responseAggregateValue <-
HashMap.lookup fieldNameKey responseAggregates
`onNothing` throw500 ("Unable to find expected aggregate " <> API.unFieldName fieldNameKey <> " in aggregates returned by Data Connector agent") -- TODO(dchambers): Add pathing information for error clarity
pure (fieldNameText, J.toEncoding responseAggregateValue)
AFOp AggregateOp {..} -> do
reshapedColumnFields <- forM _aoFields $ \(columnFieldName@(FieldName columnFieldNameText), columnField) ->
case columnField of
CFCol _column _columnType -> do
let fieldPrefix' = fieldPrefix <> prefixWith fieldName
let columnFieldNameKey = API.FieldName . getFieldNameTxt $ applyPrefix fieldPrefix' columnFieldName
responseAggregateValue <-
HashMap.lookup columnFieldNameKey responseAggregates
`onNothing` throw500 ("Unable to find expected aggregate " <> API.unFieldName columnFieldNameKey <> " in aggregates returned by Data Connector agent") -- TODO(dchambers): Add pathing information for error clarity
pure (columnFieldNameText, J.toEncoding responseAggregateValue)
CFExp txt ->
pure (columnFieldNameText, JE.text txt)
pure (fieldNameText, encodeAssocListAsObject reshapedColumnFields)
AFExp txt ->
pure (fieldNameText, JE.text txt)
pure $ encodeAssocListAsObject reshapedFields
reshapeAnnFields ::
MonadError QErr m =>
FieldPrefix ->
AnnFieldsG 'DataConnector Void v ->
HashMap API.FieldName API.FieldValue ->
m J.Encoding
reshapeAnnFields fieldNamePrefix fields responseRow = do
reshapedFields <- forM fields $ \(fieldName@(FieldName fieldNameText), field) -> do
let fieldNameKey = API.FieldName . getFieldNameTxt $ applyPrefix fieldNamePrefix fieldName
let responseField =
HashMap.lookup fieldNameKey responseRow
`onNothing` throw500 ("Unable to find expected field " <> API.unFieldName fieldNameKey <> " in row returned by Data Connector agent") -- TODO(dchambers): Add pathing information for error clarity
reshapedField <- reshapeField field responseField
pure (fieldNameText, reshapedField)
pure $ encodeAssocListAsObject reshapedFields
reshapeField ::
MonadError QErr m =>
AnnFieldG 'DataConnector Void v ->
m API.FieldValue -> -- This lookup is lazy (behind the monad) so that we can _not_ do it when we've got an AFExpression
m J.Encoding
reshapeField field responseFieldValue =
case field of
AFColumn _columnField -> do
columnFieldValue <- API.deserializeAsColumnFieldValue <$> responseFieldValue
pure $ J.toEncoding columnFieldValue
AFObjectRelation objectRelationField -> do
relationshipFieldValue <- API.deserializeAsRelationshipFieldValue <$> responseFieldValue
case relationshipFieldValue of
Left err -> throw500 $ "Found column field value where relationship field value was expected in field returned by Data Connector agent: " <> err -- TODO(dchambers): Add pathing information for error clarity
Right subqueryResponse ->
let fields = _aosFields $ _aarAnnSelect objectRelationField
in reshapeSimpleSelectRows Single fields subqueryResponse
AFArrayRelation (ASSimple simpleArrayRelationField) ->
reshapeAnnRelationSelect (reshapeSimpleSelectRows Many) simpleArrayRelationField =<< responseFieldValue
AFArrayRelation (ASAggregate aggregateArrayRelationField) ->
reshapeAnnRelationSelect reshapeTableAggregateFields aggregateArrayRelationField =<< responseFieldValue
AFExpression txt -> pure $ JE.text txt
AFNestedObject nestedObj -> do
nestedObjectFieldValue <- API.deserializeAsNestedObjFieldValue <$> responseFieldValue
case nestedObjectFieldValue of
Left err -> throw500 $ "Expected object in field returned by Data Connector agent: " <> err -- TODO(dmoverton): Add pathing information for error clarity
Right nestedResponse ->
reshapeAnnFields noPrefix (_anosFields nestedObj) nestedResponse
reshapeAnnRelationSelect ::
MonadError QErr m =>
(Fields (fieldType v) -> API.QueryResponse -> m J.Encoding) ->
AnnRelationSelectG 'DataConnector (AnnSelectG 'DataConnector fieldType v) ->
API.FieldValue ->
m J.Encoding
reshapeAnnRelationSelect reshapeFields annRelationSelect fieldValue =
case API.deserializeAsRelationshipFieldValue fieldValue of
Left err -> throw500 $ "Found column field value where relationship field value was expected in field returned by Data Connector agent: " <> err -- TODO(dchambers): Add pathing information for error clarity
Right subqueryResponse ->
let annSimpleSelect = _aarAnnSelect annRelationSelect
in reshapeFields (_asnFields annSimpleSelect) subqueryResponse