Clean Relay's code, break schema cycles, introduce Node ID V2

## Motivation

This PR rewrites most of Relay to achieve the following:
- ~~fix a bug in which the same node id could refer to two different tables in the schema~~
- remove one of the few remaining uses of the source cache in the schema building code

In doing so, it also:
- simplifies the `BackendSchema` class by removing `node` from it,
- makes it much easier for other backends to support Relay,
- documents, re-organizes, and clarifies the code.

## Description

This PR introduces a new `NodeId` version ~~, and adapts the Postgres code to always generate this V2 version~~. This new id contains the source name, in addition to the table name, in order to disambiguate similar table names across different sources (which is now possible with source customization). In doing so, it now explicitly handles that case for V1 node ids, and returns an explicit error message instead of running the risk of _silently returning the wrong information_.

Furthermore, it adapts `nodeField` to support multiple backends; most of the code was trivial to generalize, and as a result it lowers the cost of entry for other backends, that now only need to support `AFNodeId` in their translation layer.

Finally, it removes one more cycle in the schema building code, by using the same trick we used for remote relationships instead of using the memoization trick of #4576.

## Remaining work

- ~~[ ]write a Changelog entry~~
- ~~[x] adapt all tests that were asserting on an old node id~~

## Future work

This PR was adapted from its original form to avoid a breaking change: while it introduces a Node ID V2, we keep generating V1 IDs and the parser rejects V2 IDs. It will be easy to make the switch at a later data in a subsequent PR.

PR-URL: https://github.com/hasura/graphql-engine-mono/pull/4593
GitOrigin-RevId: 88e5cb91e8b0646900547fa8c7c0e1463de267a1
This commit is contained in:
Antoine Leblanc 2022-06-07 14:35:26 +01:00 committed by hasura-bot
parent 0817d46f1a
commit eaba2e08d3
27 changed files with 603 additions and 396 deletions

View File

@ -796,11 +796,13 @@ library
, Hasura.GraphQL.Schema.BoolExp
, Hasura.GraphQL.Schema.Build
, Hasura.GraphQL.Schema.Common
, Hasura.GraphQL.Schema.Introspect
, Hasura.GraphQL.Schema.Instances
, Hasura.GraphQL.Schema.Introspect
, Hasura.GraphQL.Schema.Mutation
, Hasura.GraphQL.Schema.Node
, Hasura.GraphQL.Schema.OrderBy
, Hasura.GraphQL.Schema.Postgres
, Hasura.GraphQL.Schema.Relay
, Hasura.GraphQL.Schema.Remote
, Hasura.GraphQL.Schema.RemoteRelationship
, Hasura.GraphQL.Schema.Select

View File

@ -36,7 +36,6 @@ import Hasura.RQL.Types.Function
import Hasura.RQL.Types.SchemaCache hiding (askTableInfo)
import Hasura.RQL.Types.Source (SourceInfo)
import Hasura.RQL.Types.SourceCustomization (NamingCase)
import Hasura.RQL.Types.Table
import Hasura.SQL.Backend
import Language.GraphQL.Draft.Syntax qualified as G
@ -71,7 +70,6 @@ instance BackendSchema 'BigQuery where
countTypeInput = bqCountTypeInput
aggregateOrderByCountType = BigQuery.IntegerScalarType
computedField = bqComputedField
node = bqNode
----------------------------------------------------------------
-- Top level parsers
@ -509,24 +507,3 @@ bqRemoteRelationshipField ::
m (Maybe [FieldParser n (AnnotatedField 'BigQuery)])
bqRemoteRelationshipField _remoteFieldInfo = pure Nothing
-}
-- | The 'node' root field of a Relay request. Relay is currently unsupported on BigQuery,
-- meaning this parser will never be called: any attempt to create this parser should
-- therefore fail.
bqNode ::
MonadBuildSchema 'BigQuery r m n =>
m
( Parser
'Output
n
( HashMap
(TableName 'BigQuery)
( SourceName,
SourceConfig 'BigQuery,
SelPermInfo 'BigQuery,
PrimaryKeyColumns 'BigQuery,
AnnotatedFields 'BigQuery
)
)
)
bqNode = throw500 "BigQuery does not support relay; `node` should never be exposed in the schema."

View File

@ -67,8 +67,6 @@ instance BackendSchema 'DataConnector where
error "aggregateOrderByCountType: not implemented for the Data Connector backend."
computedField =
error "computedField: not implemented for the Data Connector backend."
node =
error "node: not implemented for the Data Connector backend."
--------------------------------------------------------------------------------

View File

@ -35,7 +35,6 @@ import Hasura.RQL.IR.Insert qualified as IR
import Hasura.RQL.IR.Select qualified as IR
import Hasura.RQL.Types.Backend hiding (BackendInsert)
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.ComputedField
import Hasura.RQL.Types.Function
import Hasura.RQL.Types.Relationships.Local
@ -80,7 +79,6 @@ instance BackendSchema 'MSSQL where
countTypeInput = msCountTypeInput
aggregateOrderByCountType = MSSQL.IntegerType
computedField = msComputedField
node = msNode
----------------------------------------------------------------
@ -469,24 +467,3 @@ msRemoteRelationshipField ::
m (Maybe [FieldParser n (AnnotatedField 'MSSQL)])
msRemoteRelationshipField _remoteFieldInfo = pure Nothing
-}
-- | The 'node' root field of a Relay request. Relay is currently unsupported on MSSQL,
-- meaning this parser will never be called: any attempt to create this parser should
-- therefore fail.
msNode ::
MonadBuildSchema 'MSSQL r m n =>
m
( Parser
'Output
n
( HashMap
(TableName 'MSSQL)
( SourceName,
SourceConfig 'MSSQL,
SelPermInfo 'MSSQL,
PrimaryKeyColumns 'MSSQL,
AnnotatedFields 'MSSQL
)
)
)
msNode = throw500 "MSSQL does not support relay; `node` should never be exposed in the schema."

View File

@ -55,7 +55,6 @@ instance BackendSchema 'MySQL where
countTypeInput = mysqlCountTypeInput
aggregateOrderByCountType = error "aggregateOrderByCountType: MySQL backend does not support this operation yet."
computedField = error "computedField: MySQL backend does not support this operation yet."
node = error "node: MySQL backend does not support this operation yet."
mysqlTableArgs ::
forall r m n.

View File

@ -57,11 +57,10 @@ import Hasura.RQL.IR.Update qualified as IR
import Hasura.RQL.IR.Value qualified as IR
import Hasura.RQL.Types.Backend (Backend (..))
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.Function (FunctionInfo)
import Hasura.RQL.Types.Source
import Hasura.RQL.Types.SourceCustomization
import Hasura.RQL.Types.Table (RolePermInfo (..), SelPermInfo, TableInfo, UpdPermInfo)
import Hasura.RQL.Types.Table (RolePermInfo (..), TableInfo, UpdPermInfo)
import Hasura.SQL.Backend (BackendType (Postgres), PostgresKind (Citus, Vanilla))
import Hasura.SQL.Types
import Language.GraphQL.Draft.Syntax qualified as G
@ -97,35 +96,16 @@ class PostgresSchema (pgKind :: PostgresKind) where
m [FieldParser n (QueryDB ('Postgres pgKind) (RemoteRelationshipField IR.UnpreparedValue) (IR.UnpreparedValue ('Postgres pgKind)))]
pgkRelayExtension ::
Maybe (XRelay ('Postgres pgKind))
pgkNode ::
BS.MonadBuildSchema ('Postgres pgKind) r m n =>
m
( Parser
'Output
n
( HashMap
( TableName ('Postgres pgKind)
)
( SourceName,
SourceConfig ('Postgres pgKind),
SelPermInfo ('Postgres pgKind),
PrimaryKeyColumns ('Postgres pgKind),
AnnotatedFields ('Postgres pgKind)
)
)
)
instance PostgresSchema 'Vanilla where
pgkBuildTableRelayQueryFields = buildTableRelayQueryFields
pgkBuildFunctionRelayQueryFields = buildFunctionRelayQueryFields
pgkRelayExtension = Just ()
pgkNode = nodePG
instance PostgresSchema 'Citus where
pgkBuildTableRelayQueryFields _ _ _ _ _ = pure []
pgkBuildFunctionRelayQueryFields _ _ _ _ _ = pure []
pgkRelayExtension = Nothing
pgkNode = undefined
-- postgres schema
@ -171,7 +151,6 @@ instance
countTypeInput = countTypeInput
aggregateOrderByCountType = PG.PGInteger
computedField = computedFieldPG
node = pgkNode
backendInsertParser ::
forall pgKind m r n.

View File

@ -75,7 +75,7 @@ import Hasura.Backends.Postgres.Translate.Select.Internal.JoinTree
)
import Hasura.Backends.Postgres.Translate.Select.Internal.OrderBy (processOrderByItems)
import Hasura.Backends.Postgres.Translate.Types
import Hasura.GraphQL.Schema.Common (currentNodeIdVersion, nodeIdVersionInt)
import Hasura.GraphQL.Schema.Node (currentNodeIdVersion, nodeIdVersionInt)
import Hasura.Prelude
import Hasura.RQL.IR.BoolExp
import Hasura.RQL.IR.OrderBy (OrderByItemG (OrderByItemG, obiColumn))
@ -273,7 +273,7 @@ processAnnFields sourcePrefix fieldAlias similarArrFields annFields = do
(fieldName,)
<$> case field of
AFExpression t -> pure $ S.SELit t
AFNodeId _ tn pKeys -> pure $ mkNodeId tn pKeys
AFNodeId _ sn tn pKeys -> pure $ mkNodeId sn tn pKeys
AFColumn c -> toSQLCol c
AFObjectRelation objSel -> withWriteObjectRelation $ do
let AnnRelationSelectG relName relMapping annObjSel = objSel
@ -371,8 +371,8 @@ processAnnFields sourcePrefix fieldAlias similarArrFields annFields = do
Nothing -> sqlExp
Just (S.ColumnOp opText cExp) -> S.mkSQLOpExp opText sqlExp cExp
mkNodeId :: QualifiedTable -> PrimaryKeyColumns ('Postgres pgKind) -> S.SQLExp
mkNodeId (QualifiedObject tableSchema tableName) pkeyColumns =
mkNodeId :: SourceName -> QualifiedTable -> PrimaryKeyColumns ('Postgres pgKind) -> S.SQLExp
mkNodeId _sourceName (QualifiedObject tableSchema tableName) pkeyColumns =
let columnInfoToSQLExp pgColumnInfo =
toJSONableExp LeaveNumbersAlone (ciType pgColumnInfo) False $
S.mkQIdenExp (mkBaseTableAlias sourcePrefix) $ ciColumn pgColumnInfo

View File

@ -285,8 +285,8 @@ transformAnnFields fields = do
annotatedFields <-
fields & traverseFields \case
-- AnnFields which do not need to be transformed.
AFNodeId x qt pkeys ->
pure (AFNodeId x qt pkeys, Nothing)
AFNodeId x sn qt pkeys ->
pure (AFNodeId x sn qt pkeys, Nothing)
AFColumn c ->
pure (AFColumn c, Nothing)
AFExpression t ->

View File

@ -20,7 +20,6 @@ import Data.List.Extended (duplicates)
import Data.Text qualified as T
import Data.Text.Extended (dquoteList, (<<>))
import Hasura.Base.Error
import Hasura.GraphQL.Execute.Types
import Hasura.GraphQL.Parser.Constants qualified as G
import Hasura.GraphQL.Parser.Monad (ParseT, runSchemaT)
import Hasura.GraphQL.Schema.Common
@ -127,7 +126,6 @@ fetchRemoteSchema env manager _rscName rsDef@ValidatedRemoteSchemaDef {..} = do
-- we don't support remote schemas in Relay, but the check is
-- performed ahead of time, meaning that the value here is
-- irrelevant
soQueryType = QueryHasura,
-- doesn't apply to remote schemas
soFunctionPermsContext = FunctionPermissionsInferred,
-- we default to no permissions
@ -136,6 +134,7 @@ fetchRemoteSchema env manager _rscName rsDef@ValidatedRemoteSchemaDef {..} = do
soOptimizePermissionFilters = False
},
SchemaContext
HasuraSchema
mempty
ignoreRemoteRelationship
)

View File

@ -34,9 +34,9 @@ import Hasura.GraphQL.Schema.Common
import Hasura.GraphQL.Schema.Instances ()
import Hasura.GraphQL.Schema.Introspect
import Hasura.GraphQL.Schema.Postgres
import Hasura.GraphQL.Schema.Relay
import Hasura.GraphQL.Schema.Remote (buildRemoteParser)
import Hasura.GraphQL.Schema.RemoteRelationship
import Hasura.GraphQL.Schema.Select
import Hasura.GraphQL.Schema.Table
import Hasura.Prelude
import Hasura.RQL.IR
@ -120,7 +120,7 @@ buildGQLContext ServerConfigCtx {..} queryType sources allRemoteSchemas allActio
<$> case queryType of
QueryHasura ->
buildRoleContext
(_sccSQLGenCtx, queryType, _sccFunctionPermsCtx)
(_sccSQLGenCtx, _sccFunctionPermsCtx)
sources
allRemoteSchemas
allActionInfos
@ -133,7 +133,7 @@ buildGQLContext ServerConfigCtx {..} queryType sources allRemoteSchemas allActio
QueryRelay ->
(,mempty,G.SchemaIntrospection mempty)
<$> buildRelayRoleContext
(_sccSQLGenCtx, queryType, _sccFunctionPermsCtx)
(_sccSQLGenCtx, _sccFunctionPermsCtx)
sources
allActionInfos
customTypes
@ -157,7 +157,7 @@ buildGQLContext ServerConfigCtx {..} queryType sources allRemoteSchemas allActio
buildRoleContext ::
forall m.
(MonadError QErr m, MonadIO m) =>
(SQLGenCtx, GraphQLQueryType, FunctionPermissionsCtx) ->
(SQLGenCtx, FunctionPermissionsCtx) ->
SourceCache ->
HashMap RemoteSchemaName (RemoteSchemaCtx, MetadataObject) ->
[ActionInfo] ->
@ -174,19 +174,18 @@ buildRoleContext ::
)
buildRoleContext options sources remotes allActionInfos customTypes role remoteSchemaPermsCtx expFeatures streamingSubscriptionsCtx globalDefaultNC = do
let ( SQLGenCtx stringifyNum dangerousBooleanCollapse optimizePermissionFilters,
queryType,
functionPermsCtx
) = options
schemaOptions =
SchemaOptions
stringifyNum
dangerousBooleanCollapse
queryType
functionPermsCtx
remoteSchemaPermsCtx
optimizePermissionFilters
schemaContext =
SchemaContext
HasuraSchema
sources
(remoteRelationshipField sources (fst <$> remotes))
runMonadSchema schemaOptions schemaContext role $ do
@ -299,7 +298,7 @@ buildRoleContext options sources remotes allActionInfos customTypes role remoteS
buildRelayRoleContext ::
forall m.
(MonadError QErr m, MonadIO m) =>
(SQLGenCtx, GraphQLQueryType, FunctionPermissionsCtx) ->
(SQLGenCtx, FunctionPermissionsCtx) ->
SourceCache ->
[ActionInfo] ->
AnnotatedCustomTypes ->
@ -309,14 +308,12 @@ buildRelayRoleContext ::
m (RoleContext GQLContext)
buildRelayRoleContext options sources allActionInfos customTypes role expFeatures globalDefaultNC = do
let ( SQLGenCtx stringifyNum dangerousBooleanCollapse optimizePermissionFilters,
queryType,
functionPermsCtx
) = options
schemaOptions =
SchemaOptions
stringifyNum
dangerousBooleanCollapse
queryType
functionPermsCtx
RemoteSchemaPermsDisabled
optimizePermissionFilters
@ -325,18 +322,15 @@ buildRelayRoleContext options sources allActionInfos customTypes role expFeature
-- are not supported yet, we use `mempty` below for `RemoteSchemaMap`.
schemaContext =
SchemaContext
(RelaySchema $ nodeInterface sources)
sources
(remoteRelationshipField sources mempty)
runMonadSchema schemaOptions schemaContext role do
node <- fmap NotNamespaced <$> nodeField sources
fieldsList <- traverse (buildBackendSource buildSource) $ toList sources
-- Add node root field.
-- FIXME: for now this is PG-only. This isn't a problem yet since for now only PG supports relay.
-- To fix this, we'd need to first generalize `nodeField`.
nodeField_ <- fmap NotNamespaced <$> nodeField
let (queryPGFields', mutationFrontendFields, mutationBackendFields, subscriptionPGFields') = mconcat fieldsList
queryPGFields = nodeField_ : queryPGFields'
subscriptionPGFields = nodeField_ : subscriptionPGFields'
let (queryFields, mutationFrontendFields, mutationBackendFields, subscriptionFields) = mconcat fieldsList
allQueryFields = node : queryFields
allSubscriptionFields = node : subscriptionFields
-- Remote schema mutations aren't exposed in relay because many times it throws
-- the conflicting definitions error between the relay types like `Node`, `PageInfo` etc
@ -345,11 +339,11 @@ buildRelayRoleContext options sources allActionInfos customTypes role expFeature
mutationParserBackend <-
buildMutationParser mempty allActionInfos customTypes mutationBackendFields
subscriptionParser <-
buildSubscriptionParser subscriptionPGFields [] customTypes []
buildSubscriptionParser allSubscriptionFields [] customTypes []
queryParserFrontend <-
queryWithIntrospectionHelper queryPGFields mutationParserFrontend subscriptionParser
queryWithIntrospectionHelper allQueryFields mutationParserFrontend subscriptionParser
queryParserBackend <-
queryWithIntrospectionHelper queryPGFields mutationParserBackend subscriptionParser
queryWithIntrospectionHelper allQueryFields mutationParserBackend subscriptionParser
-- In order to catch errors early, we attempt to generate the data
-- required for introspection, which ends up doing a few correctness
@ -447,12 +441,12 @@ unauthenticatedContext allRemotes remoteSchemaPermsCtx = do
SchemaOptions
LeaveNumbersAlone -- stringifyNum doesn't apply to remotes
True -- booleanCollapse doesn't apply to remotes
QueryHasura
FunctionPermissionsInferred -- function permissions don't apply to remotes
remoteSchemaPermsCtx
False
fakeSchemaContext =
SchemaContext
HasuraSchema
mempty
ignoreRemoteRelationship
-- chosen arbitrarily to be as improbable as possible

View File

@ -46,14 +46,12 @@ import Hasura.RQL.IR.Insert qualified as IR
import Hasura.RQL.IR.Select qualified as IR
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.Column hiding (EnumValueInfo)
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.ComputedField
import Hasura.RQL.Types.Function
import Hasura.RQL.Types.Relationships.Local
import Hasura.RQL.Types.SchemaCache
import Hasura.RQL.Types.Source
import Hasura.RQL.Types.SourceCustomization (NamingCase)
import Hasura.RQL.Types.Table
import Hasura.SQL.Backend
import Hasura.Server.Types (StreamingSubscriptionsCtx)
import Language.GraphQL.Draft.Syntax qualified as G
@ -246,11 +244,6 @@ class
TableInfo b ->
m (Maybe (FieldParser n (AnnotatedField b)))
-- | The 'node' root field of a Relay request.
node ::
MonadBuildSchema b r m n =>
m (Parser 'Output n (HashMap (TableName b) (SourceName, SourceConfig b, SelPermInfo b, PrimaryKeyColumns b, AnnotatedFields b)))
type ComparisonExp b = OpExpG b (UnpreparedValue b)
-- $modelling

View File

@ -3,10 +3,13 @@
module Hasura.GraphQL.Schema.Common
( SchemaOptions (..),
SchemaContext (..),
SchemaKind (..),
RemoteRelationshipParserBuilder (..),
NodeInterfaceParserBuilder (..),
MonadBuildSchemaBase,
retrieve,
ignoreRemoteRelationship,
isHasuraSchema,
AggSelectExp,
AnnotatedField,
AnnotatedFields,
@ -24,10 +27,8 @@ module Hasura.GraphQL.Schema.Common
getTableRoles,
askTableInfo,
comparisonAggOperators,
currentNodeIdVersion,
mapField,
mkDescriptionWith,
nodeIdVersionInt,
numericAggOperators,
optionalFieldParser,
parsedSelectionsToFields,
@ -43,7 +44,6 @@ module Hasura.GraphQL.Schema.Common
)
where
import Data.Aeson qualified as J
import Data.Either (isRight)
import Data.Has
import Data.HashMap.Strict qualified as Map
@ -52,12 +52,12 @@ import Data.Text qualified as T
import Data.Text.Extended
import Hasura.Backends.Postgres.SQL.Types qualified as PG
import Hasura.Base.Error
import Hasura.GraphQL.Execute.Types qualified as ET (GraphQLQueryType)
import Hasura.GraphQL.Namespace (NamespacedField)
import Hasura.GraphQL.Parser qualified as P
import Hasura.GraphQL.Parser.Constants qualified as G
import Hasura.GraphQL.Parser.Internal.Parser qualified as P
import Hasura.GraphQL.Parser.Internal.TypeChecking qualified as P
import Hasura.GraphQL.Schema.Node
import Hasura.Prelude
import Hasura.RQL.IR qualified as IR
import Hasura.RQL.IR.BoolExp
@ -83,8 +83,6 @@ data SchemaOptions = SchemaOptions
soStringifyNum :: StringifyNumbers,
-- | should boolean fields be collapsed to True when null is given?
soDangerousBooleanCollapse :: Bool,
-- | what kind of schema is being built (Hasura or Relay)
soQueryType :: ET.GraphQLQueryType,
-- | whether function permissions should be inferred
soFunctionPermsContext :: FunctionPermissionsCtx,
-- | whether remote schema permissions are enabled
@ -95,12 +93,24 @@ data SchemaOptions = SchemaOptions
-- | Aggregation of contextual information required to build the schema.
data SchemaContext = SchemaContext
{ -- | the set of all sources (TODO: remove this?)
{ -- | the kind of schema being built
scSchemaKind :: SchemaKind,
-- | the set of all sources (TODO: remove this?)
scSourceCache :: SourceCache,
-- | how to process remote relationships
scRemoteRelationshipParserBuilder :: RemoteRelationshipParserBuilder
}
-- | The kind of schema we're building, and its associated options.
data SchemaKind
= HasuraSchema
| RelaySchema NodeInterfaceParserBuilder
isHasuraSchema :: SchemaKind -> Bool
isHasuraSchema = \case
HasuraSchema -> True
RelaySchema _ -> False
-- | The set of common constraints required to build the schema.
type MonadBuildSchemaBase r m n =
( MonadError QErr m,
@ -138,6 +148,19 @@ newtype RemoteRelationshipParserBuilder
ignoreRemoteRelationship :: RemoteRelationshipParserBuilder
ignoreRemoteRelationship = RemoteRelationshipParserBuilder $ const $ pure Nothing
-- | How to build the 'Relay' node.
--
-- Similarly to what we do for remote relationships, we pass in the context the
-- builder function required to build the 'Node' interface, in order to avoid
-- the cross-sources cycles it creates otherwise.
newtype NodeInterfaceParserBuilder = NodeInterfaceParserBuilder
{ runNodeBuilder ::
( forall r n m.
MonadBuildSchemaBase r m n =>
m (P.Parser 'P.Output n NodeMap)
)
}
-- TODO: move this to Prelude?
retrieve ::
(MonadReader r m, Has a r) =>
@ -248,23 +271,6 @@ numericAggOperators =
comparisonAggOperators :: [G.Name]
comparisonAggOperators = [$$(G.litName "max"), $$(G.litName "min")]
data NodeIdVersion
= NIVersion1
deriving (Show, Eq)
nodeIdVersionInt :: NodeIdVersion -> Int
nodeIdVersionInt NIVersion1 = 1
currentNodeIdVersion :: NodeIdVersion
currentNodeIdVersion = NIVersion1
instance J.FromJSON NodeIdVersion where
parseJSON v = do
versionInt :: Int <- J.parseJSON v
case versionInt of
1 -> pure NIVersion1
_ -> fail $ "expecting version 1 for node id, but got " <> show versionInt
mkDescriptionWith :: Maybe PG.PGDescription -> Text -> G.Description
mkDescriptionWith descM defaultTxt = G.Description $ case descM of
Nothing -> defaultTxt

View File

@ -0,0 +1,253 @@
-- | A collection of types and utilities around the @Node@ GraphQL
-- type exposed by the Relay API.
module Hasura.GraphQL.Schema.Node
( -- * Node id
NodeId (..),
V1NodeId (..),
V2NodeId (..),
-- * Node id version
NodeIdVersion,
nodeIdVersionInt,
currentNodeIdVersion,
-- * Internal relay types
NodeMap,
TableMap (..),
NodeInfo (..),
findNode,
)
where
import Data.Aeson qualified as J
import Data.Aeson.Types qualified as J
import Data.HashMap.Strict qualified as Map
import Data.Sequence qualified as Seq
import Data.Sequence.NonEmpty qualified as NESeq
import Hasura.Backends.Postgres.SQL.Types qualified as PG
import Hasura.Prelude
import Hasura.RQL.IR qualified as IR
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.Table
import Hasura.SQL.AnyBackend qualified as AB
{- Note [Relay Node Id]
~~~~~~~~~~~~~~~~~~~~~~~
Relay API
---------
The 'Node' interface in the Relay API schema has exactly one field, which
returns a non-null 'ID' value. In a backend that supports the Relay API, each
table's corresponding GraphQL object implements that interface, and provides an
@id@ field that uniuqely identifies each row of the table. See
https://relay.dev/graphql/objectidentification.htm for more details.
To uniquely identify a given row in a given table, we use two different pieces
of information:
- something that uniquely identifies the table within the schema
- something that uniquely identifies the row within the table
Both V1 and V2 (of this particular API, not of the engine, see 'NodeIdVersion')
use the same data to uniquely identify the row within the table: a list of
values that map to the table's primary keys, in order. Where they differentiate
is on how they identify the table within the schema:
- V1 only used a Postgres table name;
- V2 uses a source name, and a backend-agnostic table name
For now, we still only emit and accept V1 ids: switching to emitting V2 node ids
will be a breaking change that will we do soon. We will continue to accept V1
node ids after that change, meaning we still to resolve them; in practice, that
means iterating over all the Postgres sources, until we find one that has a
table with the given name. If we find more than one, then we fail, to avoid
having to pick a random one (and potentially silently return wrong results.)
Id format
---------
All the required information is encoded into a unique node id using the
following pipeline:
values <-> JSON array <-> bytestring <-> base64 string
In v1, the content of the JSON array was:
[ 1 -- JSON number: version number
, "public" -- JSON string: Postgres schema name
, "foo" -- JSON string: Postgres table name
, ... -- arbitrary JSON values: values for each primary key, in order
]
As of v2, the content of the JSON array is as follows:
[ 2 -- JSON number: version number
, "default" -- JSON string: source name
, "postgres" -- JSON string: backend type
, { "schema: "public" -- arbitrary JSON value: table name in that backend
, "name": "foo"
}
, ... -- arbitrary JSON values: values for each primary key, in order
]
Encoding and decoding
---------------------
The encoding of a given row's id is performed in each backend's translation
layer, as crafting the row's id requires extracting information out of the
database (the primary key values). Selecting the 'id' field of a compatible
table will yield an 'AFNodeId' field in the IR (see Hasura.RQL.IR.Select), that
each compatible backend will then interpret appropriately.
Decoding, however, does not require introspecting the database, and is performed
at parsing time, so that we can select the corresponing table row. See
'nodeField' in 'Relay.hs' for more information.
-}
--------------------------------------------------------------------------------
-- Node id
data NodeId
= NodeIdV1 V1NodeId
| NodeIdV2 (AB.AnyBackend V2NodeId)
-- | V1 format of a node.
--
-- This id does NOT uniquely identify the table properly, as it only knows the
-- table's name, but doesn't store a source name.
data V1NodeId = V1NodeId
{ _ni1Table :: PG.QualifiedTable,
_ni1Columns :: NESeq.NESeq J.Value
}
-- | V2 format of a node.
--
-- Uniquely identifies a table with source name and table name, and uniquely
-- identifies a row within that table with a list of primary key values.
data V2NodeId b = V2NodeId
{ _ni2Source :: SourceName,
_ni2Table :: TableName b,
_ni2Columns :: !(NESeq.NESeq J.Value)
}
instance J.FromJSON NodeId where
parseJSON = J.withArray "node id" \array -> case toList array of
[] -> fail "unexpected GUID format, found empty list"
J.Number 1 : rest -> NodeIdV1 <$> parseNodeIdV1 rest
J.Number n : _ -> fail $ "unsupported GUID version: " <> show n
_ -> fail "unexpected GUID format, needs to start with a version number"
parseNodeIdV1 :: [J.Value] -> J.Parser V1NodeId
parseNodeIdV1 (schemaValue : nameValue : firstColumn : remainingColumns) =
V1NodeId
<$> (PG.QualifiedObject <$> J.parseJSON schemaValue <*> J.parseJSON nameValue)
<*> pure (firstColumn NESeq.:<|| Seq.fromList remainingColumns)
parseNodeIdV1 _ = fail "GUID version 1: expecting schema name, table name and at least one column value"
--------------------------------------------------------------------------------
-- Node id version
-- | Enum representing the supported versions of the API.
data NodeIdVersion
= NIVersion1
| NIVersion2
deriving (Show, Eq)
nodeIdVersionInt :: NodeIdVersion -> Int
nodeIdVersionInt = \case
NIVersion1 -> 1
NIVersion2 -> 2
currentNodeIdVersion :: NodeIdVersion
currentNodeIdVersion = NIVersion1
--------------------------------------------------------------------------------
-- Internal relay types
{- Note [Internal Relay HashMap]
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Parsing the Node interface
--------------------------
When selecting a node in the schema, the user can use GraphQL fragments to
select different rows based on what table the node id maps to. For instance, a
Relay query could look like this (assuming that there are corresponding tables
"album" and "track" in the schema, possibly in different sources):
query {
node(id: "base64idhere") {
... on album {
albumName
}
... on track {
trackName
}
}
}
What that means is that the parser for the 'Node' interface needs to delegate to
*every table parser*, to deal with all possible cases. In practice, we use the
'selectionSetInterface' combinator (from Hasura.GraphQL.Parser.Internal.Parser):
we give it a list of all the parsers, and it in turn applies all of them, and
gives us the result for each possible table:
- if the table was "album", the parsed result is: ...
- if the table was "track", the parsed result is: ...
The parser for the interface itself cannot know what the actual underlying table
is: that's determined by the node id, which is not something inherent to the
interface! Consequently, what the parser for the interface returns is a
container, that to every supported table in the schema, associates the
corresponding parser output; the node *field* can then use that map and the node
id it got as an argument to extract the relevant information out of said
container.
The 'NodeMap' container
-----------------------
To avoid having to do extra lookups, we also store in that container additional
information about the table: permissions for the current role, connection
information... so that the field, by simply doing a lookup based on the node id,
can have all the information it needs to craft a corresponding query.
In practice: the value we store in our container is a 'NodeInfo' (see
below). Our container, in turn, isn't a 'HashMap' from "unique table identifier"
to 'NodeInfo'; the problem is that not all sources have the same backend type,
meaning that the "unique table identifier" would need to be a _hetereogeneous_
key type. This can be achieved with a dependent map (such as
Data.Dependent.Map.DMap), but is extremely cumbersome. Instead, our overall
container, 'NodeMap', is two layers of 'HashMap': to a source name, we associate
a "backend-erased" 'TableMap' which, in turn, for the corresponding backend,
associates to a table name the corresponding 'NodeInfo'.
Module structure
----------------
Ideally, none of those types should be exported: they are used in the return
type of 'nodeInteface', but consumed immediately by 'nodeField' (see both in
Relay.hs), and they could therefore be purely internal... except for the fact
that 'Common.hs' needs to know about the NodeMap, which is why it is defined
here instead of being an implementation detail of 'Relay.hs'.
-}
type NodeMap = HashMap SourceName (AB.AnyBackend TableMap)
-- | All the information required to craft a query to a row pointed to by a
-- 'NodeId'.
data NodeInfo b = NodeInfo
{ nvSourceConfig :: SourceConfig b,
nvSelectPermissions :: SelPermInfo b,
nvPrimaryKeys :: PrimaryKeyColumns b,
nvAnnotatedFields :: IR.AnnFieldsG b (IR.RemoteRelationshipField IR.UnpreparedValue) (IR.UnpreparedValue b)
}
newtype TableMap b = TableMap (HashMap (TableName b) (NodeInfo b))
-- | Given a source name and table name, peform the double lookup within a
-- 'NodeMap'.
findNode :: forall b. Backend b => SourceName -> TableName b -> NodeMap -> Maybe (NodeInfo b)
findNode sourceName tableName nodeMap = do
anyTableMap <- Map.lookup sourceName nodeMap
TableMap tableMap <- AB.unpackAnyBackend @b anyTableMap
Map.lookup tableName tableMap

View File

@ -0,0 +1,222 @@
{-# LANGUAGE TemplateHaskellQuotes #-}
-- | This module only exposes one function, 'nodeField', which is used at the
-- root level of the schema to create the 'node' field in the Relay API schema.
module Hasura.GraphQL.Schema.Relay
( nodeInterface,
nodeField,
)
where
import Control.Lens hiding (index)
import Data.Aeson qualified as J
import Data.Aeson.Extended qualified as J
import Data.Align (align)
import Data.Has
import Data.HashMap.Strict.Extended qualified as Map
import Data.Parser.JSONPath
import Data.Sequence.NonEmpty qualified as NESeq
import Data.Text qualified as T
import Data.Text.Extended
import Data.These (partitionThese)
import Hasura.Base.Error
import Hasura.GraphQL.Parser (Kind (..), Parser)
import Hasura.GraphQL.Parser qualified as P
import Hasura.GraphQL.Parser.Class
import Hasura.GraphQL.Parser.Constants qualified as G
import Hasura.GraphQL.Parser.Internal.Parser qualified as P
import Hasura.GraphQL.Schema.Backend
import Hasura.GraphQL.Schema.Common
import Hasura.GraphQL.Schema.Instances ()
import Hasura.GraphQL.Schema.Node
import Hasura.GraphQL.Schema.Select
import Hasura.GraphQL.Schema.Table
import Hasura.Prelude
import Hasura.RQL.IR qualified as IR
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.SchemaCache hiding (askTableInfo)
import Hasura.RQL.Types.Source
import Hasura.RQL.Types.SourceCustomization
import Hasura.RQL.Types.Table
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.SQL.Backend
import Language.GraphQL.Draft.Syntax qualified as G
-- | Constructs the parser for the node interface.
--
-- As mentioned in Note [Internal Relay HashMap], this function must parse an
-- incoming query for ANY potential matching table. Its resulting parser returns
-- a 'NodeMap': a container that, to a source name and a table name, associates
-- both the parsed fields and all the relevant table information required to
-- craft a request.
nodeInterface :: SourceCache -> NodeInterfaceParserBuilder
nodeInterface sourceCache = NodeInterfaceParserBuilder $ memoizeOn 'nodeInterface () do
let idDescription = G.Description "A globally unique identifier"
idField = P.selection_ G._id (Just idDescription) P.identifier
nodeInterfaceDescription = G.Description "An object with globally unique ID"
tCase <- asks getter
tables :: [Parser 'Output n (SourceName, AB.AnyBackend TableMap)] <-
catMaybes . concat <$> for (Map.toList sourceCache) \(sourceName, anySourceInfo) ->
AB.dispatchAnyBackend @BackendSchema anySourceInfo \(sourceInfo :: SourceInfo b) ->
for (Map.toList $ takeValidTables $ _siTables sourceInfo) \(tableName, tableInfo) -> runMaybeT do
tablePkeyColumns <- hoistMaybe $ tableInfo ^? tiCoreInfo . tciPrimaryKey . _Just . pkColumns
selectPermissions <- MaybeT $ tableSelectPermissions tableInfo
annotatedFieldsParser <-
MaybeT $
P.withTypenameCustomization
(mkCustomizedTypename (_scTypeNames $ _siCustomization sourceInfo) tCase)
(tableSelectionSet sourceInfo tableInfo)
pure $
annotatedFieldsParser <&> \fields ->
( sourceName,
AB.mkAnyBackend $
TableMap $
Map.singleton tableName $
NodeInfo (_siConfiguration sourceInfo) selectPermissions tablePkeyColumns fields
)
pure $
Map.fromListWith fuseAnyMaps
<$> P.selectionSetInterface
G._Node
(Just nodeInterfaceDescription)
[idField]
tables
where
-- this can only ever fail if somehow, within the same source, we ran into
-- two tables of a different type b; in other words, it is impossible.
fuseAnyMaps :: AB.AnyBackend TableMap -> AB.AnyBackend TableMap -> AB.AnyBackend TableMap
fuseAnyMaps m1 m2 =
AB.composeAnyBackend @Backend fuseMaps m1 m2 $
error "panic: two tables of a different backend type within the same source"
fuseMaps :: forall b. Backend b => TableMap b -> TableMap b -> AB.AnyBackend TableMap
fuseMaps (TableMap m1) (TableMap m2) = AB.mkAnyBackend @b $ TableMap $ Map.union m1 m2
-- | Creates a field parser for the top-level "node" field in the QueryRoot.
--
-- It exepcts one argument, the node id. It looks for the targeted table in the
-- 'NodeMap' returned by 'nodeInterface', and, if successful, attempts to craft
-- a corresponding 'QueryRootField' that will extract the requested row.
nodeField ::
forall m n r.
SourceCache ->
MonadBuildSchemaBase r m n =>
m (P.FieldParser n (IR.QueryRootField IR.UnpreparedValue))
nodeField sourceCache = do
let idDescription = G.Description "A globally unique id"
idArgument = P.field G._id (Just idDescription) P.identifier
stringifyNum <- retrieve soStringifyNum
nodeObject <-
retrieve scSchemaKind >>= \case
HasuraSchema -> throw500 "internal error: the node field should only be built for the Relay schema"
RelaySchema nodeBuilder -> runNodeBuilder nodeBuilder
pure $
P.subselection G._node Nothing idArgument nodeObject `P.bindField` \(ident, parseds) -> do
nodeId <- parseNodeId ident
case nodeId of
NodeIdV1 (V1NodeId tableName pKeys) -> do
-- Node id V1.
--
-- We don't have the source name in a V1 node; we attempt all of them
-- and pick the first one we find; there is a risk we might pick the
-- wrong one if two tables with the same name exist in different
-- sources! It is, however, unlikely; the engine emits V2 IDs, meaning
-- if ever encounter a V1 ID it means it has been manually entered bya
-- user, saved from an older version of the engine?
let matchingTables = flip mapMaybe (Map.keys sourceCache) \sourceName ->
(sourceName,) <$> findNode @('Postgres 'Vanilla) sourceName tableName parseds
case matchingTables of
[(sourceName, nodeValue)] -> createRootField stringifyNum sourceName tableName nodeValue pKeys
[] -> throwInvalidNodeId $ "no such table found: " <>> tableName
l ->
throwInvalidNodeId $
"this V1 node id matches more than one table across different sources: " <> tableName
<<> " exists in sources "
<> commaSeparated (fst <$> l)
NodeIdV2 nodev2 ->
-- Node id V2.
--
-- We have the source name and table name, we can extract the relevant
-- info directly.
AB.dispatchAnyBackend @Backend nodev2 \(V2NodeId sourceName tableName pKeys :: V2NodeId b) -> do
nodeValue <-
findNode @b sourceName tableName parseds
`onNothing` throwInvalidNodeId ("no table " <> tableName <<> " found in source " <>> sourceName)
createRootField stringifyNum sourceName tableName nodeValue pKeys
where
throwInvalidNodeId :: Text -> n a
throwInvalidNodeId t = withPath (++ [Key "args", Key "id"]) $ parseError $ "invalid node id: " <> t
parseNodeId :: Text -> n NodeId
parseNodeId = either (throwInvalidNodeId . T.pack) pure . J.eitherDecode . base64Decode
-- Given all the node id information about a table, and the extracted
-- 'NodeInfo', craft the top-level query. This relies on the assumption
-- that all backends that support relay use the same IR for single row
-- selection.
createRootField ::
Backend b =>
StringifyNumbers ->
SourceName ->
TableName b ->
NodeInfo b ->
NESeq.NESeq J.Value ->
n (IR.QueryRootField IR.UnpreparedValue)
createRootField stringifyNum sourceName tableName (NodeInfo sourceConfig perms pKeys fields) columnValues = do
whereExp <- buildNodeIdBoolExp columnValues pKeys
pure $
IR.RFDB sourceName $
AB.mkAnyBackend $
IR.SourceConfigWith sourceConfig Nothing $
IR.QDBR $
IR.QDBSingleRow $
IR.AnnSelectG
{ IR._asnFields = fields,
IR._asnFrom = IR.FromTable tableName,
IR._asnPerm = tablePermissionsInfo perms,
IR._asnArgs =
IR.SelectArgs
{ IR._saWhere = Just whereExp,
IR._saOrderBy = Nothing,
IR._saLimit = Nothing,
IR._saOffset = Nothing,
IR._saDistinct = Nothing
},
IR._asnStrfyNum = stringifyNum
}
-- Craft the 'where' condition of the query by making an `AEQ` entry for
-- each primary key. This might fail if the given node id doesn't exactly
-- have a valid entry for each primary key.
buildNodeIdBoolExp ::
Backend b =>
NESeq.NESeq J.Value ->
NESeq.NESeq (ColumnInfo b) ->
n (IR.AnnBoolExp b (IR.UnpreparedValue b))
buildNodeIdBoolExp columnValues pkeyColumns = do
let firstPkColumn NESeq.:<|| remainingPkColumns = pkeyColumns
firstColumnValue NESeq.:<|| remainingColumns = columnValues
(nonAlignedPkColumns, nonAlignedColumnValues, alignedTuples) =
partitionThese $ toList $ align remainingPkColumns remainingColumns
unless (null nonAlignedPkColumns) $
throwInvalidNodeId $
"primary key columns " <> dquoteList (map ciColumn nonAlignedPkColumns) <> " are missing"
unless (null nonAlignedColumnValues) $
throwInvalidNodeId $
"unexpected column values " <> J.encodeToStrictText nonAlignedColumnValues
let allTuples = (firstPkColumn, firstColumnValue) : alignedTuples
IR.BoolAnd <$> for allTuples \(columnInfo, columnValue) -> do
let columnType = ciType columnInfo
parsedValue <-
parseScalarValueColumnType columnType columnValue `onLeft` \e ->
parseErrorWith ParseFailed $ "value of column " <> ciColumn columnInfo <<> " in node id: " <> qeError e
pure $
IR.BoolFld $
IR.AVColumn
columnInfo
[IR.AEQ True $ IR.UVParameter Nothing $ ColumnValue columnType parsedValue]

View File

@ -9,7 +9,6 @@ import Data.HashMap.Strict.Extended qualified as Map
import Data.List.NonEmpty qualified as NE
import Data.Text.Extended
import Hasura.Base.Error
import Hasura.GraphQL.Execute.Types qualified as ET
import Hasura.GraphQL.Parser
import Hasura.GraphQL.Parser qualified as P
import Hasura.GraphQL.Parser.Constants qualified as G
@ -43,10 +42,10 @@ remoteRelationshipField ::
RemoteRelationshipParserBuilder
remoteRelationshipField sourceCache remoteSchemaCache = RemoteRelationshipParserBuilder
\RemoteFieldInfo {..} -> runMaybeT do
queryType <- retrieve soQueryType
queryType <- retrieve scSchemaKind
-- https://github.com/hasura/graphql-engine/issues/5144
-- The above issue is easily fixable by removing the following guard
guard $ queryType == ET.QueryHasura
guard $ isHasuraSchema queryType
case _rfiRHS of
RFISource anyRemoteSourceFieldInfo ->
dispatchAnyBackend @BackendSchema anyRemoteSourceFieldInfo \remoteSourceFieldInfo -> do

View File

@ -24,33 +24,25 @@ module Hasura.GraphQL.Schema.Select
tablePermissionsInfo,
tableSelectionSet,
tableSelectionList,
nodePG,
nodeField,
)
where
import Control.Lens hiding (index)
import Data.Aeson qualified as J
import Data.Aeson.Extended qualified as J
import Data.Aeson.Internal qualified as J
import Data.Align (align)
import Data.ByteString.Lazy qualified as BL
import Data.Has
import Data.HashMap.Strict.Extended qualified as Map
import Data.Int (Int64)
import Data.List.NonEmpty qualified as NE
import Data.Parser.JSONPath
import Data.Sequence qualified as Seq
import Data.Sequence.NonEmpty qualified as NESeq
import Data.Text qualified as T
import Data.Text.Extended
import Data.These (partitionThese)
import Data.Traversable (mapAccumL)
import Hasura.Backends.Postgres.SQL.Types qualified as PG
import Hasura.Backends.Postgres.Types.ComputedField qualified as PG
import Hasura.Backends.Postgres.Types.Function qualified as PG
import Hasura.Base.Error
import Hasura.GraphQL.Execute.Types qualified as ET
import Hasura.GraphQL.Parser
( FieldParser,
InputFieldsParser,
@ -80,7 +72,6 @@ import Hasura.RQL.Types.SchemaCache hiding (askTableInfo)
import Hasura.RQL.Types.Source
import Hasura.RQL.Types.SourceCustomization
import Hasura.RQL.Types.Table
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.SQL.Backend
import Hasura.Server.Utils (executeJSONPath)
import Language.GraphQL.Draft.Syntax qualified as G
@ -365,7 +356,13 @@ tableSelectionSet ::
m (Maybe (Parser 'Output n (AnnotatedFields b)))
tableSelectionSet sourceInfo tableInfo = runMaybeT do
_selectPermissions <- MaybeT $ tableSelectPermissions tableInfo
lift $ memoizeOn 'tableSelectionSet (_siName sourceInfo, tableName) do
schemaKind <- lift $ retrieve scSchemaKind
-- If this check fails, it means we're attempting to build a Relay schema, but
-- the current backend b does't support Relay; rather than returning an
-- incomplete selection set, we fail early and return 'Nothing'. This check
-- must happen first, since we can't memoize a @Maybe Parser@.
guard $ isHasuraSchema schemaKind || isJust (relayExtension @b)
lift $ memoizeOn 'tableSelectionSet (sourceName, tableName) do
tableGQLName <- getTableGQLName tableInfo
objectTypename <- P.mkTypename tableGQLName
let xRelay = relayExtension @b
@ -376,7 +373,7 @@ tableSelectionSet sourceInfo tableInfo = runMaybeT do
concat
<$> for
tableFields
(fieldSelection sourceInfo tableName tableInfo tablePkeyColumns)
(fieldSelection sourceInfo tableName tableInfo)
-- We don't check *here* that the subselection set is non-empty,
-- even though the GraphQL specification requires that it is (see
@ -386,14 +383,13 @@ tableSelectionSet sourceInfo tableInfo = runMaybeT do
-- required, meaning that not having this check here does not allow
-- for the construction of invalid queries.
queryType <- retrieve soQueryType
case (queryType, tablePkeyColumns, xRelay) of
case (schemaKind, tablePkeyColumns, xRelay) of
-- A relay table
(ET.QueryRelay, Just pkeyColumns, Just xRelayInfo) -> do
(RelaySchema nodeBuilder, Just pkeyColumns, Just xRelayInfo) -> do
let nodeIdFieldParser =
P.selection_ G._id Nothing P.identifier $> IR.AFNodeId xRelayInfo tableName pkeyColumns
P.selection_ G._id Nothing P.identifier $> IR.AFNodeId xRelayInfo sourceName tableName pkeyColumns
allFieldParsers = fieldParsers <> [nodeIdFieldParser]
nodeInterface <- node @b
nodeInterface <- runNodeBuilder nodeBuilder
pure $
P.selectionSetObject objectTypename description allFieldParsers [nodeInterface]
<&> parsedSelectionsToFields IR.AFExpression
@ -402,6 +398,7 @@ tableSelectionSet sourceInfo tableInfo = runMaybeT do
P.selectionSetObject objectTypename description fieldParsers []
<&> parsedSelectionsToFields IR.AFExpression
where
sourceName = _siName sourceInfo
tableName = tableInfoName tableInfo
tableCoreInfo = _tiCoreInfo tableInfo
@ -1060,61 +1057,56 @@ fieldSelection ::
SourceInfo b ->
TableName b ->
TableInfo b ->
Maybe (PrimaryKeyColumns b) ->
FieldInfo b ->
m [FieldParser n (AnnotatedField b)]
fieldSelection sourceInfo table tableInfo maybePkeyColumns = \case
fieldSelection sourceInfo table tableInfo = \case
FIColumn columnInfo ->
maybeToList <$> runMaybeT do
queryType <- retrieve soQueryType
schemaKind <- retrieve scSchemaKind
let fieldName = ciName columnInfo
if fieldName == G._id && queryType == ET.QueryRelay
then do
xRelayInfo <- hoistMaybe $ relayExtension @b
pkeyColumns <- hoistMaybe maybePkeyColumns
pure $
P.selection_ fieldName Nothing P.identifier
$> IR.AFNodeId xRelayInfo table pkeyColumns
else do
let columnName = ciColumn columnInfo
selectPermissions <- MaybeT $ tableSelectPermissions tableInfo
guard $ columnName `Map.member` spiCols selectPermissions
let caseBoolExp = join $ Map.lookup columnName (spiCols selectPermissions)
caseBoolExpUnpreparedValue =
(fmap . fmap) partialSQLExpToUnpreparedValue <$> caseBoolExp
pathArg = scalarSelectionArgumentsParser $ ciType columnInfo
-- In an inherited role, when a column is part of all the select
-- permissions which make up the inherited role then the nullability
-- of the field is determined by the nullability of the DB column
-- otherwise it is marked as nullable explicitly, ignoring the column's
-- nullability. We do this because
-- in multiple roles we execute an SQL query like:
--
-- select
-- (case when (P1 or P2) then addr else null end) as addr,
-- (case when P2 then phone else null end) as phone
-- from employee
-- where (P1 or P2)
--
-- In the above example, P(n) is a predicate configured for a role
--
-- NOTE: https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf
-- The above is the paper which talks about the idea of cell-level
-- authorization and multiple roles. The paper says that we should only
-- allow the case analysis only on nullable columns.
nullability = ciIsNullable columnInfo || isJust caseBoolExp
field <- lift $ columnParser (ciType columnInfo) (G.Nullability nullability)
pure $
P.selection fieldName (ciDescription columnInfo) pathArg field
<&> IR.mkAnnColumnField (ciColumn columnInfo) (ciType columnInfo) caseBoolExpUnpreparedValue
-- If the field name is 'id' and we're building a schema for the Relay
-- API, Node's id field will take precedence; consequently we simply
-- ignore the original.
guard $ isHasuraSchema schemaKind || fieldName /= G._id
let columnName = ciColumn columnInfo
selectPermissions <- MaybeT $ tableSelectPermissions tableInfo
guard $ columnName `Map.member` spiCols selectPermissions
let caseBoolExp = join $ Map.lookup columnName (spiCols selectPermissions)
caseBoolExpUnpreparedValue =
(fmap . fmap) partialSQLExpToUnpreparedValue <$> caseBoolExp
pathArg = scalarSelectionArgumentsParser $ ciType columnInfo
-- In an inherited role, when a column is part of all the select
-- permissions which make up the inherited role then the nullability
-- of the field is determined by the nullability of the DB column
-- otherwise it is marked as nullable explicitly, ignoring the column's
-- nullability. We do this because
-- in multiple roles we execute an SQL query like:
--
-- select
-- (case when (P1 or P2) then addr else null end) as addr,
-- (case when P2 then phone else null end) as phone
-- from employee
-- where (P1 or P2)
--
-- In the above example, P(n) is a predicate configured for a role
--
-- NOTE: https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf
-- The above is the paper which talks about the idea of cell-level
-- authorization and multiple roles. The paper says that we should only
-- allow the case analysis only on nullable columns.
nullability = ciIsNullable columnInfo || isJust caseBoolExp
field <- lift $ columnParser (ciType columnInfo) (G.Nullability nullability)
pure $
P.selection fieldName (ciDescription columnInfo) pathArg field
<&> IR.mkAnnColumnField (ciColumn columnInfo) (ciType columnInfo) caseBoolExpUnpreparedValue
FIRelationship relationshipInfo ->
concat . maybeToList <$> relationshipField sourceInfo table relationshipInfo
FIComputedField computedFieldInfo ->
maybeToList <$> computedField sourceInfo computedFieldInfo table tableInfo
FIRemoteRelationship remoteFieldInfo -> do
queryType <- retrieve soQueryType
case (queryType, _rfiRHS remoteFieldInfo) of
(ET.QueryRelay, RFISchema _) ->
schemaKind <- retrieve scSchemaKind
case (schemaKind, _rfiRHS remoteFieldInfo) of
(RelaySchema _, RFISchema _) ->
-- Remote schemas aren't currently supported in Relay, and we therefore
-- cannot include remote relationships to them while building a
-- Relay-specific schema: attempting to do so would raise an error, as
@ -1321,8 +1313,7 @@ relationshipField sourceInfo table ri = runMaybeT do
remoteAggField <- lift $ selectTableAggregate sourceInfo otherTableInfo relAggFieldName relAggDesc
remoteConnectionField <- runMaybeT $ do
-- Parse array connection field only for relay schema
queryType <- retrieve soQueryType
guard $ queryType == ET.QueryRelay
RelaySchema _ <- retrieve scSchemaKind
_xRelayInfo <- hoistMaybe $ relayExtension @b
pkeyColumns <-
MaybeT $
@ -1583,185 +1574,3 @@ tablePermissionsInfo selectPermissions =
{ IR._tpFilter = fmap partialSQLExpToUnpreparedValue <$> spiFilter selectPermissions,
IR._tpLimit = spiLimit selectPermissions
}
------------------------ Node interface from Relay ---------------------------
{- Note [Relay Node Id]
~~~~~~~~~~~~~~~~~~~~~~~
The 'Node' interface in Relay schema has exactly one field which returns
a non-null 'ID' value. Each table object type in Relay schema should implement
'Node' interface to provide global object identification.
See https://relay.dev/graphql/objectidentification.htm for more details.
To identify each row in a table, we need to encode the table information
(schema and name) and primary key column values in the 'Node' id.
Node id data:
-------------
We are using JSON format for encoding and decoding the node id. The JSON
schema looks like following
'[<version-integer>, "<table-schema>", "<table-name>", "column-1", "column-2", ... "column-n"]'
It is represented in the type @'NodeId'. The 'version-integer' represents the JSON
schema version to enable any backward compatibility if it is broken in upcoming versions.
The stringified JSON is Base64 encoded and sent to client. Also the same
base64 encoded JSON string is accepted for 'node' field resolver's 'id' input.
-}
data V1NodeId = V1NodeId
{ _nidTable :: !(TableName ('Postgres 'Vanilla)),
_nidColumns :: !(NESeq.NESeq J.Value)
}
deriving (Show, Eq)
-- | The Relay 'Node' inteface's 'id' field value.
-- See Note [Relay Node id].
data NodeId
= NodeIdV1 !V1NodeId
deriving (Show, Eq)
instance J.FromJSON NodeId where
parseJSON v = do
valueList <- J.parseJSON v
case valueList of
[] -> fail "unexpected GUID format, found empty list"
J.Number 1 : rest -> NodeIdV1 <$> parseNodeIdV1 rest
J.Number n : _ -> fail $ "unsupported GUID version: " <> show n
_ -> fail "unexpected GUID format, needs to start with a version number"
where
parseNodeIdV1 (schemaValue : (nameValue : (firstColumn : remainingColumns))) =
V1NodeId
<$> (PG.QualifiedObject <$> J.parseJSON schemaValue <*> J.parseJSON nameValue)
<*> pure (firstColumn NESeq.:<|| Seq.fromList remainingColumns)
parseNodeIdV1 _ = fail "GUID version 1: expecting schema name, table name and at least one column value"
throwInvalidNodeId :: MonadParse n => Text -> n a
throwInvalidNodeId t = parseError $ "the node id is invalid: " <> t
-- | The 'node' root field of a Relay request.
nodePG ::
forall m n r.
MonadBuildSchema ('Postgres 'Vanilla) r m n =>
m
( P.Parser
'Output
n
( HashMap
(TableName ('Postgres 'Vanilla))
( SourceName,
SourceConfig ('Postgres 'Vanilla),
SelPermInfo ('Postgres 'Vanilla),
PrimaryKeyColumns ('Postgres 'Vanilla),
AnnotatedFields ('Postgres 'Vanilla)
)
)
)
nodePG = memoizeOn 'nodePG () do
let idDescription = G.Description "A globally unique identifier"
idField = P.selection_ G._id (Just idDescription) P.identifier
nodeInterfaceDescription = G.Description "An object with globally unique ID"
sources <- retrieve scSourceCache
tCase <- asks getter
tables <-
Map.fromList . catMaybes <$> sequence do
(sourceName, sourceInfo) <- Map.toList sources
pgInfo <- maybeToList $ AB.unpackAnyBackend @('Postgres 'Vanilla) sourceInfo
tableName <- Map.keys $ takeValidTables $ _siTables pgInfo
pure $ runMaybeT do
tableInfo <- lift $ askTableInfo pgInfo tableName
tablePkeyColumns <- hoistMaybe $ tableInfo ^? tiCoreInfo . tciPrimaryKey . _Just . pkColumns
selectPermissions <- MaybeT $ tableSelectPermissions tableInfo
annotatedFieldsParser <-
MaybeT $
P.withTypenameCustomization
(mkCustomizedTypename (_scTypeNames $ _siCustomization pgInfo) tCase)
(tableSelectionSet pgInfo tableInfo)
pure
( tableName,
(sourceName,_siConfiguration pgInfo,selectPermissions,tablePkeyColumns,)
<$> annotatedFieldsParser
)
pure $
P.selectionSetInterface
G._Node
(Just nodeInterfaceDescription)
[idField]
tables
nodeField ::
forall m n r.
MonadBuildSchema ('Postgres 'Vanilla) r m n =>
m (P.FieldParser n (IR.QueryRootField IR.UnpreparedValue))
nodeField = do
let idDescription = G.Description "A globally unique id"
idArgument = P.field G._id (Just idDescription) P.identifier
stringifyNum <- retrieve soStringifyNum
nodeObject <- node
return $
P.subselection G._node Nothing idArgument nodeObject
`P.bindField` \(ident, parseds) -> do
NodeIdV1 (V1NodeId table columnValues) <- parseNodeId ident
(source, sourceConfig, perms, pkeyColumns, fields) <-
onNothing (Map.lookup table parseds) $
withArgsPath $ throwInvalidNodeId $ "the table " <>> ident
whereExp <- buildNodeIdBoolExp columnValues pkeyColumns
return $
IR.RFDB source $
AB.mkAnyBackend $
IR.SourceConfigWith sourceConfig Nothing $
IR.QDBR $
IR.QDBSingleRow $
IR.AnnSelectG
{ IR._asnFields = fields,
IR._asnFrom = IR.FromTable table,
IR._asnPerm = tablePermissionsInfo perms,
IR._asnArgs =
IR.SelectArgs
{ IR._saWhere = Just whereExp,
IR._saOrderBy = Nothing,
IR._saLimit = Nothing,
IR._saOffset = Nothing,
IR._saDistinct = Nothing
},
IR._asnStrfyNum = stringifyNum
}
where
parseNodeId :: Text -> n NodeId
parseNodeId =
either (withArgsPath . throwInvalidNodeId . T.pack) pure . J.eitherDecode . base64Decode
withArgsPath = withPath (++ [Key "args", Key "id"])
buildNodeIdBoolExp ::
NESeq.NESeq J.Value ->
NESeq.NESeq (ColumnInfo ('Postgres 'Vanilla)) ->
n (IR.AnnBoolExp ('Postgres 'Vanilla) (IR.UnpreparedValue ('Postgres 'Vanilla)))
buildNodeIdBoolExp columnValues pkeyColumns = do
let firstPkColumn NESeq.:<|| remainingPkColumns = pkeyColumns
firstColumnValue NESeq.:<|| remainingColumns = columnValues
(nonAlignedPkColumns, nonAlignedColumnValues, alignedTuples) =
partitionThese $ toList $ align remainingPkColumns remainingColumns
unless (null nonAlignedPkColumns) $
throwInvalidNodeId $
"primary key columns " <> dquoteList (map ciColumn nonAlignedPkColumns) <> " are missing"
unless (null nonAlignedColumnValues) $
throwInvalidNodeId $
"unexpected column values " <> J.encodeToStrictText nonAlignedColumnValues
let allTuples = (firstPkColumn, firstColumnValue) : alignedTuples
flip onLeft (parseErrorWith ParseFailed . qeError) $
runExcept $
fmap IR.BoolAnd $
for allTuples $ \(columnInfo, columnValue) -> do
let modifyErrFn t =
"value of column " <> ciColumn columnInfo
<<> " in node id: " <> t
pgColumnType = ciType columnInfo
pgValue <- modifyErr modifyErrFn $ parseScalarValueColumnType pgColumnType columnValue
let unpreparedValue = IR.UVParameter Nothing $ ColumnValue pgColumnType pgValue
pure $ IR.BoolFld $ IR.AVColumn columnInfo [IR.AEQ True unpreparedValue]

View File

@ -577,10 +577,10 @@ data AnnFieldG (b :: BackendType) (r :: Type) v
= AFColumn (AnnColumnField b v)
| AFObjectRelation (ObjectRelationSelectG b r v)
| AFArrayRelation (ArraySelectG b r v)
| AFComputedField (XComputedField b) !ComputedFieldName !(ComputedFieldSelect b r v)
| AFComputedField (XComputedField b) ComputedFieldName (ComputedFieldSelect b r v)
| -- | A remote relationship field
AFRemote (RemoteRelationshipSelect b r)
| AFNodeId (XRelay b) !(TableName b) !(PrimaryKeyColumns b)
| AFNodeId (XRelay b) SourceName (TableName b) (PrimaryKeyColumns b)
| AFExpression Text
deriving stock (Functor, Foldable, Traversable)

View File

@ -124,6 +124,7 @@ buildQueryParsers introspection = do
HasuraCase :: NamingCase,
undefined :: SchemaOptions,
SchemaContext
HasuraSchema
mempty
ignoreRemoteRelationship
)

View File

@ -15,10 +15,9 @@ import Data.Aeson.Internal (JSONPath)
import Data.Has (Has (..))
import Data.Text qualified as T
import Hasura.Base.Error (Code, QErr)
import Hasura.GraphQL.Execute.Types (GraphQLQueryType (..))
import Hasura.GraphQL.Parser.Class (MonadParse (..), MonadSchema (..))
import Hasura.GraphQL.Parser.Schema (MkTypename (..))
import Hasura.GraphQL.Schema.Common (SchemaContext (..), SchemaOptions (..), ignoreRemoteRelationship)
import Hasura.GraphQL.Schema.Common (SchemaContext (..), SchemaKind (..), SchemaOptions (..), ignoreRemoteRelationship)
import Hasura.Prelude
import Hasura.RQL.Types.Common (StringifyNumbers (LeaveNumbersAlone))
import Hasura.RQL.Types.Function (FunctionPermissionsCtx (..))
@ -64,7 +63,6 @@ instance Has SchemaOptions SchemaEnvironment where
SchemaOptions
{ soStringifyNum = LeaveNumbersAlone,
soDangerousBooleanCollapse = False,
soQueryType = QueryHasura,
soFunctionPermsContext = FunctionPermissionsInferred,
soRemoteSchemaPermsCtx = RemoteSchemaPermsDisabled,
soOptimizePermissionFilters = False
@ -78,7 +76,8 @@ instance Has SchemaContext SchemaEnvironment where
getter =
const
SchemaContext
{ scSourceCache = notImplemented "scSourceCache",
{ scSchemaKind = HasuraSchema,
scSourceCache = mempty,
scRemoteRelationshipParserBuilder = ignoreRemoteRelationship
}

View File

@ -16,4 +16,4 @@ response:
- extensions:
path: "$.selectionSet.node"
code: validation-failed
message: the node id is invalid
message: invalid node id

View File

@ -18,5 +18,5 @@ response:
- extensions:
path: $.selectionSet.node.args.id
code: validation-failed
message: 'the node id is invalid: Error in $: GUID version 1: expecting schema
message: 'invalid node id: Error in $: GUID version 1: expecting schema
name, table name and at least one column value'

View File

@ -18,5 +18,5 @@ response:
- extensions:
path: "$.selectionSet.node.args.id"
code: validation-failed
message: 'the node id is invalid: Error in $: Failed reading: not a valid json value
message: 'invalid node id: Error in $: Failed reading: not a valid json value
at ''­©Ý¢'''

View File

@ -16,4 +16,4 @@ response:
- extensions:
path: $.selectionSet.node.args.id
code: validation-failed
message: 'the node id is invalid: Error in $: unsupported GUID version: 2.0'
message: 'invalid node id: Error in $: unsupported GUID version: 2.0'

View File

@ -16,6 +16,6 @@ query:
response:
errors:
- extensions:
path: "$.selectionSet.node"
path: "$.selectionSet.node.args.id"
code: validation-failed
message: 'the node id is invalid: primary key columns "last_name" are missing'
message: 'invalid node id: primary key columns "last_name" are missing'

View File

@ -16,5 +16,5 @@ response:
- extensions:
path: $.selectionSet.node.args.id
code: validation-failed
message: 'the node id is invalid: Error in $: parsing [] failed, expected Array,
message: 'invalid node id: Error in $: parsing node id failed, expected Array,
but encountered Object'

View File

@ -18,5 +18,5 @@ response:
- extensions:
path: $.selectionSet.node.args.id
code: validation-failed
message: 'the node id is invalid: Error in $: unexpected GUID format, needs to
message: 'invalid node id: Error in $: unexpected GUID format, needs to
start with a version number'

View File

@ -16,6 +16,6 @@ query:
response:
errors:
- extensions:
path: "$.selectionSet.node"
path: "$.selectionSet.node.args.id"
code: validation-failed
message: 'the node id is invalid: unexpected column values [2]'
message: 'invalid node id: unexpected column values [2]'