graphql-engine/server/src-lib/Hasura/RQL/DDL/Schema/Cache.hs

1244 lines
58 KiB
Haskell
Raw Normal View History

{-# LANGUAGE Arrows #-}
{-# LANGUAGE OverloadedLabels #-}
{-# LANGUAGE UndecidableInstances #-}
-- | Top-level functions concerned specifically with operations on the schema cache, such as
-- rebuilding it from the catalog and incorporating schema changes. See the module documentation for
-- "Hasura.RQL.DDL.Schema" for more details.
--
-- __Note__: this module is __mutually recursive__ with other @Hasura.RQL.DDL.Schema.*@ modules, which
-- both define pieces of the implementation of building the schema cache and define handlers that
-- trigger schema cache rebuilds.
module Hasura.RQL.DDL.Schema.Cache
( RebuildableSchemaCache,
lastBuiltSchemaCache,
buildRebuildableSchemaCache,
buildRebuildableSchemaCacheWithReason,
CacheRWT,
runCacheRWT,
mkBooleanPermissionMap,
)
where
import Control.Arrow.Extended
import Control.Arrow.Interpret
import Control.Lens hiding ((.=))
import Control.Monad.Trans.Control (MonadBaseControl)
import Control.Retry qualified as Retry
import Data.Aeson
import Data.Either (isLeft)
import Data.Environment qualified as Env
import Data.HashMap.Strict.Extended qualified as M
import Data.HashMap.Strict.InsOrd.Extended qualified as OMap
import Data.HashSet qualified as HS
import Data.Proxy
import Data.Set qualified as S
import Data.Text.Extended
import Hasura.Base.Error
import Hasura.GraphQL.Schema (buildGQLContext)
import Hasura.GraphQL.Schema.NamingCase
import Hasura.Incremental qualified as Inc
import Hasura.Logging
import Hasura.Metadata.Class
import Hasura.Prelude
import Hasura.RQL.DDL.Action
import Hasura.RQL.DDL.CustomTypes
import Hasura.RQL.DDL.EventTrigger (MonadEventLogCleanup (..), buildEventTriggerInfo)
import Hasura.RQL.DDL.InheritedRoles (resolveInheritedRole)
import Hasura.RQL.DDL.RemoteRelationship (CreateRemoteSchemaRemoteRelationship (..), PartiallyResolvedSource (..), buildRemoteFieldInfo, getRemoteSchemaEntityJoinColumns)
import Hasura.RQL.DDL.ScheduledTrigger
import Hasura.RQL.DDL.Schema.Cache.Common
import Hasura.RQL.DDL.Schema.Cache.Dependencies
import Hasura.RQL.DDL.Schema.Cache.Fields
import Hasura.RQL.DDL.Schema.Cache.Permission
import Hasura.RQL.DDL.Schema.Function
import Hasura.RQL.DDL.Schema.Table
import Hasura.RQL.Types.Action
import Hasura.RQL.Types.Allowlist
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.CustomTypes
import Hasura.RQL.Types.Endpoint
import Hasura.RQL.Types.EventTrigger
import Hasura.RQL.Types.Eventing.Backend
import Hasura.RQL.Types.Function
import Hasura.RQL.Types.Metadata hiding (fmFunction, tmTable)
import Hasura.RQL.Types.Metadata.Backend
import Hasura.RQL.Types.Metadata.Object
import Hasura.RQL.Types.Network
import Hasura.RQL.Types.OpenTelemetry
import Hasura.RQL.Types.QueryCollection
import Hasura.RQL.Types.Relationships.Remote
import Hasura.RQL.Types.Roles
import Hasura.RQL.Types.ScheduledTrigger
import Hasura.RQL.Types.SchemaCache
import Hasura.RQL.Types.SchemaCache.Build
import Hasura.RQL.Types.SchemaCache.Instances ()
import Hasura.RQL.Types.SchemaCacheTypes
import Hasura.RQL.Types.Source
import Hasura.RQL.Types.SourceCustomization
import Hasura.RQL.Types.Table
scaffolding for remote-schemas module The main aim of the PR is: 1. To set up a module structure for 'remote-schemas' package. 2. Move parts by the remote schema codebase into the new module structure to validate it. ## Notes to the reviewer Why a PR with large-ish diff? 1. We've been making progress on the MM project but we don't yet know long it is going to take us to get to the first milestone. To understand this better, we need to figure out the unknowns as soon as possible. Hence I've taken a stab at the first two items in the [end-state](https://gist.github.com/0x777/ca2bdc4284d21c3eec153b51dea255c9) document to figure out the unknowns. Unsurprisingly, there are a bunch of issues that we haven't discussed earlier. These are documented in the 'open questions' section. 1. The diff is large but that is only code moved around and I've added a section that documents how things are moved. In addition, there are fair number of PR comments to help with the review process. ## Changes in the PR ### Module structure Sets up the module structure as follows: ``` Hasura/ RemoteSchema/ Metadata/ Types.hs SchemaCache/ Types.hs Permission.hs RemoteRelationship.hs Build.hs MetadataAPI/ Types.hs Execute.hs ``` ### 1. Types representing metadata are moved Types that capture metadata information (currently scattered across several RQL modules) are moved into `Hasura.RemoteSchema.Metadata.Types`. - This new module only depends on very 'core' modules such as `Hasura.Session` for the notion of roles and `Hasura.Incremental` for `Cacheable` typeclass. - The requirement on database modules is avoided by generalizing the remote schemas metadata to accept an arbitrary 'r' for a remote relationship definition. ### 2. SchemaCache related types and build logic have been moved Types that represent remote schemas information in SchemaCache are moved into `Hasura.RemoteSchema.SchemaCache.Types`. Similar to `H.RS.Metadata.Types`, this module depends on 'core' modules except for `Hasura.GraphQL.Parser.Variable`. It has something to do with remote relationships but I haven't spent time looking into it. The validation of 'remote relationships to remote schema' is also something that needs to be looked at. Rips out the logic that builds remote schema's SchemaCache information from the monolithic `buildSchemaCacheRule` and moves it into `Hasura.RemoteSchema.SchemaCache.Build`. Further, the `.SchemaCache.Permission` and `.SchemaCache.RemoteRelationship` have been created from existing modules that capture schema cache building logic for those two components. This was a fair amount of work. On main, currently remote schema's SchemaCache information is built in two phases - in the first phase, 'permissions' and 'remote relationships' are ignored and in the second phase they are filled in. While remote relationships can only be resolved after partially resolving sources and other remote schemas, the same isn't true for permissions. Further, most of the work that is done to resolve remote relationships can be moved to the first phase so that the second phase can be a very simple traversal. This is the approach that was taken - resolve permissions and as much as remote relationships information in the first phase. ### 3. Metadata APIs related types and build logic have been moved The types that represent remote schema related metadata APIs and the execution logic have been moved to `Hasura.RemoteSchema.MetadataAPI.Types` and `.Execute` modules respectively. ## Open questions: 1. `Hasura.RemoteSchema.Metadata.Types` is so called because I was hoping that all of the metadata related APIs of remote schema can be brought in at `Hasura.RemoteSchema.Metadata.API`. However, as metadata APIs depended on functions from `SchemaCache` module (see [1](https://github.com/hasura/graphql-engine-mono/blob/ceba6d62264603ee5d279814677b29bcc43ecaea/server/src-lib/Hasura/RQL/DDL/RemoteSchema.hs#L55) and [2](https://github.com/hasura/graphql-engine-mono/blob/ceba6d62264603ee5d279814677b29bcc43ecaea/server/src-lib/Hasura/RQL/DDL/RemoteSchema.hs#L91), it made more sense to create a separate top-level module for `MetadataAPI`s. Maybe we can just have `Hasura.RemoteSchema.Metadata` and get rid of the extra nesting or have `Hasura.RemoteSchema.Metadata.{Core,Permission,RemoteRelationship}` if we want to break them down further. 1. `buildRemoteSchemas` in `H.RS.SchemaCache.Build` has the following type: ```haskell buildRemoteSchemas :: ( ArrowChoice arr, Inc.ArrowDistribute arr, ArrowWriter (Seq CollectedInfo) arr, Inc.ArrowCache m arr, MonadIO m, HasHttpManagerM m, Inc.Cacheable remoteRelationshipDefinition, ToJSON remoteRelationshipDefinition, MonadError QErr m ) => Env.Environment -> ( (Inc.Dependency (HashMap RemoteSchemaName Inc.InvalidationKey), OrderedRoles), [RemoteSchemaMetadataG remoteRelationshipDefinition] ) `arr` HashMap RemoteSchemaName (PartiallyResolvedRemoteSchemaCtxG remoteRelationshipDefinition, MetadataObject) ``` Note the dependence on `CollectedInfo` which is defined as ```haskell data CollectedInfo = CIInconsistency InconsistentMetadata | CIDependency MetadataObject -- ^ for error reporting on missing dependencies SchemaObjId SchemaDependency deriving (Eq) ``` this pretty much means that remote schemas is dependent on types from databases, actions, .... How do we fix this? Maybe introduce a typeclass such as `ArrowCollectRemoteSchemaDependencies` which is defined in `Hasura.RemoteSchema` and then implemented in graphql-engine? 1. The dependency on `buildSchemaCacheFor` in `.MetadataAPI.Execute` which has the following signature: ```haskell buildSchemaCacheFor :: (QErrM m, CacheRWM m, MetadataM m) => MetadataObjId -> MetadataModifier -> ``` This can be easily resolved if we restrict what the metadata APIs are allowed to do. Currently, they operate in an unfettered access to modify SchemaCache (the `CacheRWM` constraint): ```haskell runAddRemoteSchema :: ( QErrM m, CacheRWM m, MonadIO m, HasHttpManagerM m, MetadataM m, Tracing.MonadTrace m ) => Env.Environment -> AddRemoteSchemaQuery -> m EncJSON ``` This should instead be changed to restrict remote schema APIs to only modify remote schema metadata (but has access to the remote schemas part of the schema cache), this dependency is completely removed. ```haskell runAddRemoteSchema :: ( QErrM m, MonadIO m, HasHttpManagerM m, MonadReader RemoteSchemasSchemaCache m, MonadState RemoteSchemaMetadata m, Tracing.MonadTrace m ) => Env.Environment -> AddRemoteSchemaQuery -> m RemoteSchemeMetadataObjId ``` The idea is that the core graphql-engine would call these functions and then call `buildSchemaCacheFor`. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6291 GitOrigin-RevId: 51357148c6404afe70219afa71bd1d59bdf4ffc6
2022-10-21 06:13:07 +03:00
import Hasura.RemoteSchema.Metadata
import Hasura.RemoteSchema.SchemaCache
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.SQL.Backend
import Hasura.SQL.BackendMap (BackendMap)
import Hasura.SQL.BackendMap qualified as BackendMap
import Hasura.SQL.Tag
import Hasura.Server.Migrate.Version
import Hasura.Server.Types
import Hasura.Session
import Hasura.Tracing qualified as Tracing
import Language.GraphQL.Draft.Syntax qualified as G
import Network.HTTP.Client.Manager (HasHttpManagerM (..))
{- Note [Roles Inheritance]
~~~~~~~~~~~~~~~~~~~~~~~~~~~
Roles may have parent roles defined from which they can inherit permission and this is
called as roles inheritance. Roles which have parents can also be parents of other roles.
So, cycle in roles should be disallowed and this is done in the `orderRoles` function.
When the metadata contains a permission for a role for a entity, then it will override the
inherited permission, if any.
Roles inheritance work differently for different features:
1. Select permissions
~~~~~~~~~~~~~~~~~~~~~
See note [Inherited roles architecture for read queries]
2. Mutation permissions and remote schema permissions
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
For mutation and remote schema permissions, an inherited role can only inherit permission
from its parent roles when the relevant parts of the permissions are equal i.e. the non-relevant
parts are discarded for the equality, for example, in two remote schema permissions the order
of the fields in an Object type is discarded.
When an inherited role cannot inherit permission from its parents due to a conflict, then we mark
the inherited role and the entity (remote schema or table) combination as inconsistent in the metadata.
3. Actions and Custom function permissions
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Currently, actions and custom function permissions can be thought of as a boolean. Either a role has
permission to the entity or it doesn't, so in these cases there's no possiblity of a conflict. An inherited
role will have access to the action/function if any one of the parents have permission to access the
action/function.
-}
buildRebuildableSchemaCache ::
Logger Hasura ->
Env.Environment ->
Metadata ->
CacheBuild RebuildableSchemaCache
buildRebuildableSchemaCache =
buildRebuildableSchemaCacheWithReason CatalogSync
buildRebuildableSchemaCacheWithReason ::
BuildReason ->
Logger Hasura ->
Env.Environment ->
Metadata ->
CacheBuild RebuildableSchemaCache
buildRebuildableSchemaCacheWithReason reason logger env metadata = do
result <-
flip runReaderT reason $
Inc.build (buildSchemaCacheRule logger env) (metadata, initialInvalidationKeys)
pure $ RebuildableSchemaCache (Inc.result result) initialInvalidationKeys (Inc.rebuildRule result)
newtype CacheRWT m a
= -- The CacheInvalidations component of the state could actually be collected using WriterT, but
-- WriterT implementations prior to transformers-0.5.6.0 (which added
-- Control.Monad.Trans.Writer.CPS) are leaky, and we dont have that yet.
CacheRWT (StateT (RebuildableSchemaCache, CacheInvalidations) m a)
deriving
( Functor,
Applicative,
Monad,
MonadIO,
MonadReader r,
MonadError e,
UserInfoM,
HasHttpManagerM,
MonadMetadataStorage,
MonadMetadataStorageQueryAPI,
Tracing.MonadTrace,
HasServerConfigCtx,
MonadBase b,
MonadBaseControl b
)
instance (MonadEventLogCleanup m) => MonadEventLogCleanup (CacheRWT m) where
runLogCleaner conf = lift $ runLogCleaner conf
generateCleanupSchedules sourceInfo triggerName cleanupConfig = lift $ generateCleanupSchedules sourceInfo triggerName cleanupConfig
runCacheRWT ::
Functor m =>
RebuildableSchemaCache ->
CacheRWT m a ->
m (a, RebuildableSchemaCache, CacheInvalidations)
runCacheRWT cache (CacheRWT m) =
runStateT m (cache, mempty) <&> \(v, (newCache, invalidations)) -> (v, newCache, invalidations)
instance MonadTrans CacheRWT where
lift = CacheRWT . lift
instance (Monad m) => CacheRM (CacheRWT m) where
askSchemaCache = CacheRWT $ gets (lastBuiltSchemaCache . (^. _1))
instance
( MonadIO m,
MonadError QErr m,
HasHttpManagerM m,
MonadResolveSource m,
HasServerConfigCtx m
) =>
CacheRWM (CacheRWT m)
where
buildSchemaCacheWithOptions buildReason invalidations metadata = CacheRWT do
(RebuildableSchemaCache lastBuiltSC invalidationKeys rule, oldInvalidations) <- get
let metadataVersion = scMetadataResourceVersion lastBuiltSC
newInvalidationKeys = invalidateKeys invalidations invalidationKeys
result <-
lift $
runCacheBuildM $
flip runReaderT buildReason $
Inc.build rule (metadata, newInvalidationKeys)
let schemaCache = (Inc.result result) {scMetadataResourceVersion = metadataVersion}
prunedInvalidationKeys = pruneInvalidationKeys schemaCache newInvalidationKeys
!newCache = RebuildableSchemaCache schemaCache prunedInvalidationKeys (Inc.rebuildRule result)
!newInvalidations = oldInvalidations <> invalidations
put (newCache, newInvalidations)
where
-- Prunes invalidation keys that no longer exist in the schema to avoid leaking memory by
-- hanging onto unnecessary keys.
pruneInvalidationKeys schemaCache = over ikRemoteSchemas $ M.filterWithKey \name _ ->
-- see Note [Keep invalidation keys for inconsistent objects]
name `elem` getAllRemoteSchemas schemaCache
setMetadataResourceVersionInSchemaCache resourceVersion = CacheRWT $ do
(rebuildableSchemaCache, invalidations) <- get
put
( rebuildableSchemaCache
{ lastBuiltSchemaCache =
(lastBuiltSchemaCache rebuildableSchemaCache)
{ scMetadataResourceVersion = Just resourceVersion
}
},
invalidations
)
-- | Generate health checks related cache from sources metadata
buildHealthCheckCache :: Sources -> SourceHealthCheckCache
buildHealthCheckCache sources =
catMaybes $ M.fromList $ map (second mkSourceHealthCheck) (OMap.toList sources)
where
mkSourceHealthCheck :: BackendSourceMetadata -> Maybe BackendSourceHealthCheckInfo
mkSourceHealthCheck (BackendSourceMetadata sourceMetadata) =
AB.traverseBackend @Backend sourceMetadata mkSourceHealthCheckBackend
mkSourceHealthCheckBackend :: SourceMetadata b -> Maybe (SourceHealthCheckInfo b)
mkSourceHealthCheckBackend sourceMetadata =
let sourceName = _smName sourceMetadata
connection = _smConfiguration sourceMetadata
healthCheck = _smHealthCheckConfig sourceMetadata
in SourceHealthCheckInfo sourceName connection <$> healthCheck
-- | Generate cache of source connection details so that we can ping sources for
-- attribution
buildSourcePingCache :: Sources -> SourcePingCache
buildSourcePingCache sources =
M.fromList $ map (second mkSourcePing) (OMap.toList sources)
where
mkSourcePing :: BackendSourceMetadata -> BackendSourcePingInfo
mkSourcePing (BackendSourceMetadata sourceMetadata) =
AB.mapBackend sourceMetadata mkSourcePingBackend
mkSourcePingBackend :: SourceMetadata b -> SourcePingInfo b
mkSourcePingBackend sourceMetadata =
let sourceName = _smName sourceMetadata
connection = _smConfiguration sourceMetadata
in SourcePingInfo sourceName connection
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
{- Note [Avoiding GraphQL schema rebuilds when changing irrelevant Metadata]
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
There are many Metadata operations that don't influence the GraphQL schema. So
we should be caching its construction.
The `Hasura.Incremental` framework allows us to cache such constructions:
whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and
`b` the output, we can use the `Inc.cache` combinator to obtain a new arrow
which is only re-executed when the input `a` changes in a material way. To test
this, `a` needs an `Eq` instance.
We can't simply apply `Inc.cache` to the GraphQL schema cache building phase
(`buildGQLContext`), because the inputs (components of `BuildOutputs` such as
`SourceCache`) don't have an `Eq` instance.
So the purpose of `buildOutputsAndSchema` is that we cach already at an earlier
point, encompassing more computation. The Metadata and invalidation keys (which
have `Eq` instances) are used as a caching key, and `Inc.cache` can be applied
to the whole sequence of steps.
But because of the all-or-nothing nature of caching, it's important that
`buildOutputsAndSchema` is re-run as little as possible. So the exercise
becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as
many Metadata operations as possible can be handled outside of this codepath
that produces a GraphQL schema.
-}
buildSchemaCacheRule ::
-- Note: by supplying BuildReason via MonadReader, it does not participate in caching, which is
-- what we want!
( ArrowChoice arr,
Inc.ArrowDistribute arr,
Inc.ArrowCache m arr,
MonadIO m,
MonadBaseControl IO m,
MonadError QErr m,
MonadReader BuildReason m,
HasHttpManagerM m,
MonadResolveSource m,
HasServerConfigCtx m
) =>
Logger Hasura ->
Env.Environment ->
(Metadata, InvalidationKeys) `arr` SchemaCache
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys) -> do
invalidationKeysDep <- Inc.newDependency -< invalidationKeys
metadataDefaults <- bindA -< askMetadataDefaults
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
let metadata@Metadata {..} = overrideMetadataDefaults metadataNoDefaults metadataDefaults
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
metadataDep <- Inc.newDependency -< metadata
(inconsistentObjects, (resolvedOutputs, dependencyInconsistentObjects, resolvedDependencies), ((adminIntrospection, gqlContext, gqlContextUnauth, inconsistentRemoteSchemas), (relayContext, relayContextUnauth))) <-
Inc.cache buildOutputsAndSchema -< (metadataDep, invalidationKeysDep)
let (resolvedEndpoints, endpointCollectedInfo) = runIdentity $ runWriterT $ buildRESTEndpoints _metaQueryCollections (OMap.elems _metaRestEndpoints)
(cronTriggersMap, cronTriggersCollectedInfo) = runIdentity $ runWriterT $ buildCronTriggers (OMap.elems _metaCronTriggers)
(openTelemetryInfo, openTelemetryCollectedInfo) = runIdentity $ runWriterT $ buildOpenTelemetry _metaOpenTelemetryConfig
duplicateVariables :: EndpointMetadata a -> Bool
duplicateVariables m = any ((> 1) . length) $ group $ sort $ catMaybes $ splitPath Just (const Nothing) (_ceUrl m)
endpointObjId :: EndpointMetadata q -> MetadataObjId
endpointObjId md = MOEndpoint (_ceName md)
endpointObject :: EndpointMetadata q -> MetadataObject
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
endpointObject md = MetadataObject (endpointObjId md) (toJSON $ OMap.lookup (_ceName md) _metaRestEndpoints)
listedQueryObjects :: (CollectionName, ListedQuery) -> MetadataObject
listedQueryObjects (cName, lq) = MetadataObject (MOQueryCollectionsQuery cName lq) (toJSON lq)
-- Cases of urls that generate invalid segments:
hasInvalidSegments :: EndpointMetadata query -> Bool
hasInvalidSegments m = any (`elem` ["", ":"]) (splitPath id id (_ceUrl m))
ceUrlTxt = toTxt . _ceUrl
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
endpoints = buildEndpointsTrie (M.elems resolvedEndpoints)
duplicateF md = DuplicateRestVariables (ceUrlTxt md) (endpointObject md)
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
duplicateRestVariables = map duplicateF $ filter duplicateVariables (M.elems resolvedEndpoints)
invalidF md = InvalidRestSegments (ceUrlTxt md) (endpointObject md)
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
invalidRestSegments = map invalidF $ filter hasInvalidSegments (M.elems resolvedEndpoints)
ambiguousF' ep = MetadataObject (endpointObjId ep) (toJSON ep)
ambiguousF mds = AmbiguousRestEndpoints (commaSeparated $ map _ceUrl mds) (map ambiguousF' mds)
ambiguousRestEndpoints = map (ambiguousF . S.elems . snd) $ ambiguousPathsGrouped endpoints
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
inlinedAllowlist = inlineAllowlist _metaQueryCollections _metaAllowlist
globalAllowLists = HS.toList . iaGlobal $ inlinedAllowlist
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
-- Endpoints don't generate any dependencies
endpointInconsistencies = either id absurd <$> toList endpointCollectedInfo
-- Cron triggers don't generate any dependencies
cronTriggersInconsistencies = either id absurd <$> toList cronTriggersCollectedInfo
-- OpenTelemerty doesn't generate any dependencies
openTelemetryInconsistencies = either id absurd <$> toList openTelemetryCollectedInfo
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
inconsistentQueryCollections <- bindA -< do getInconsistentQueryCollections adminIntrospection _metaQueryCollections listedQueryObjects endpoints globalAllowLists
returnA
-<
SchemaCache
{ scSources = _boSources resolvedOutputs,
scActions = _boActions resolvedOutputs,
-- TODO this is not the right value: we should track what part of the schema
-- we can stitch without consistencies, I think.
scRemoteSchemas = fmap fst (_boRemoteSchemas resolvedOutputs), -- remoteSchemaMap
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
scAllowlist = inlinedAllowlist,
-- , scCustomTypes = _boCustomTypes resolvedOutputs
Decouple `Analyse` and `OpenAPI` from remote schema introspection and internal execution details. ### Motivation #2338 introduced a way to validate REST queries against the metadata after a change, to properly report any inconsistency that would emerge from a change in the underlying structure of our schema. However, the way this was done was quite complex and error-prone. Namely: we would use the generated schema parsers to statically execute an introspection query, similar to the one we use for remote schemas, then parse the resulting bytestring as it were coming from a remote schema. This led to several issues: the code was using remote schema primitives, and was associated with remote schema code, despite being unrelated, which led to absurd situations like creating fake `Variable`s whose type was also their name. A lot of the code had to deal with the fact that we might fail to re-parse our own schema. Additionally, some of it was dead code, that for some reason GHC did not warn about? But more fundamentally, this architecture decision creates a dependency between unrelated pieces of the engine: modifying the internal processing of root fields or the introspection of remote schemas now risks impacting the unrelated `OpenAPI` feature. ### Description This PR decouples that process from the remote schema introspection logic and from the execution engine by making `Analyse` and `OpenAPI` work on the generic `G.SchemaIntrospection` instead. To accomplish this, it: - adds `GraphQL.Parser.Schema.Convert`, to convert from our "live" schema back to a flat `SchemaIntrospection` - persists in the schema cache the `admin` introspection generated when building the schema, and uses it both for validation and for generating the `OpenAPI`. ### Known issues and limitations This adds a bit of memory pressure to the engine, as we persist the entire schema in the schema cache. This might be acceptable in the short-term, but we have several potential ideas going forward should this be a problem: - cache the result of `Analyze`: when it becomes possible to build the `OpenAPI` purely with the result of `Analyze` without any additional schema information, then we could cache that instead, reducing the footprint - caching the `OpenAPI`: if it doesn't need to change every time the endpoint is queried, then it should be possible to cache the entire `OpenAPI` object instead of the schema - cache a copy of the `FieldParsers` used to generate the schema: as those are persisted through the GraphQL `Context`, and are the only input required to generate the `Schema`, making them accessible in the schema cache would allow us to have the exact same feature with no additional memory cost, at the price of a slightly slower and more complicated process (need to rebuild the `Schema` every time we query the OpenAPI endpoint) - cache nothing at all, and rebuild the admin schema from scratch every time. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/3962 Co-authored-by: paritosh-08 <85472423+paritosh-08@users.noreply.github.com> GitOrigin-RevId: a8b9808170b231fdf6787983b4a9ed286cde27e0
2022-03-22 10:36:39 +03:00
scAdminIntrospection = adminIntrospection,
scGQLContext = gqlContext,
scUnauthenticatedGQLContext = gqlContextUnauth,
scRelayContext = relayContext,
scUnauthenticatedRelayContext = relayContextUnauth,
-- , scGCtxMap = gqlSchema
-- , scDefaultRemoteGCtx = remoteGQLSchema
scDepMap = resolvedDependencies,
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
scCronTriggers = cronTriggersMap,
scEndpoints = endpoints,
scInconsistentObjs =
inconsistentObjects
<> dependencyInconsistentObjects
<> toList inconsistentRemoteSchemas
<> duplicateRestVariables
<> invalidRestSegments
<> ambiguousRestEndpoints
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
<> endpointInconsistencies
<> cronTriggersInconsistencies
<> openTelemetryInconsistencies
<> inconsistentQueryCollections,
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
scApiLimits = _metaApiLimits,
scMetricsConfig = _metaMetricsConfig,
scMetadataResourceVersion = Nothing,
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
scSetGraphqlIntrospectionOptions = _metaSetGraphqlIntrospectionOptions,
scTlsAllowlist = networkTlsAllowlist _metaNetwork,
scQueryCollections = _metaQueryCollections,
scBackendCache = _boBackendCache resolvedOutputs,
server: Simplify `BuildOutputs` A bunch of configurations are retrieved from the Metadata, then stored in the `BuildOutputs` structure, only to then be forwarded to the `SchemaCache`, with extremely little processing in between. So this simplifies the build pipeline for some parts of the metadata: just construct those things from `Metadata` directly, and store them in the `SchemaCache` without any intermediate container. Why did we have the detour via `BuildOutputs` in the first place? Parts of the Metadata (codified by `MetadataObjId`) can generate _metadata inconsistencies_ and/or _schema dependencies_, which are related. - Metadata inconsistencies are warnings that we show to the user, indicating that there's something wrong with their configuration, and they have to fix it. - Schema dependencies are an internal mechanism that allow us to build a consistent view of the world. For instance, if we have a relationship from DB tables `books` to `authors`, but the `authors` table is inconsistent (e.g. it doesn't exist in the DB), then we have schema dependencies indicating that. The job of `resolveDependencies` is to then drop the relationship, so that we can at least generate a legal GraphQL schema for `books`. If we never generate a schema dependency for a certain fragment of Metadata, then there is no reason to call `resolveDependencies` on it, and so there is no reason to store it in `BuildOutputs`. --- The starting point that allows this refactor is to apply Metadata defaults before it reaches `buildAndCollectInfo`, so that metadata-with-defaults can be used elsewhere. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6609 GitOrigin-RevId: df0c4a7ff9451e10e02a40bf26304b26584ba483
2022-11-15 15:02:55 +03:00
scSourceHealthChecks = buildHealthCheckCache _metaSources,
scSourcePingConfig = buildSourcePingCache _metaSources,
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
scOpenTelemetryConfig = openTelemetryInfo
}
where
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
-- See Note [Avoiding GraphQL schema rebuilds when changing irrelevant Metadata]
buildOutputsAndSchema = proc (metadataDep, invalidationKeysDep) -> do
(outputs, collectedInfo) <- runWriterA buildAndCollectInfo -< (metadataDep, invalidationKeysDep)
let (inconsistentObjects, unresolvedDependencies) = partitionEithers $ toList collectedInfo
out2@(resolvedOutputs, _dependencyInconsistentObjects, _resolvedDependencies) <- resolveDependencies -< (outputs, unresolvedDependencies)
out3 <-
bindA
-< do
cxt <- askServerConfigCtx
buildGQLContext
cxt
(_boSources resolvedOutputs)
(_boRemoteSchemas resolvedOutputs)
(_boActions resolvedOutputs)
(_boCustomTypes resolvedOutputs)
returnA -< (inconsistentObjects, out2, out3)
resolveBackendInfo' ::
forall arr m b.
( BackendMetadata b,
ArrowChoice arr,
Inc.ArrowCache m arr,
Inc.ArrowDistribute arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
MonadIO m,
HasHttpManagerM m
) =>
(BackendConfigWrapper b, Inc.Dependency (BackendMap BackendInvalidationKeysWrapper)) `arr` BackendCache
resolveBackendInfo' = proc (backendConfigWrapper, backendInvalidationMap) -> do
let backendInvalidationKeys =
Inc.selectMaybeD #unBackendInvalidationKeysWrapper $
BackendMap.lookupD @b backendInvalidationMap
backendInfo <- resolveBackendInfo @b logger -< (backendInvalidationKeys, unBackendConfigWrapper backendConfigWrapper)
returnA -< BackendMap.singleton (BackendInfoWrapper @b backendInfo)
resolveBackendCache ::
forall arr m.
( ArrowChoice arr,
Inc.ArrowCache m arr,
Inc.ArrowDistribute arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
MonadIO m,
HasHttpManagerM m
) =>
(Inc.Dependency (BackendMap BackendInvalidationKeysWrapper), [AB.AnyBackend BackendConfigWrapper]) `arr` BackendCache
resolveBackendCache = proc (backendInvalidationMap, backendConfigs) -> do
case backendConfigs of
[] -> returnA -< mempty
(anyBackendConfig : backendConfigs') -> do
backendInfo <-
AB.dispatchAnyBackendArrow @BackendMetadata @HasTag resolveBackendInfo' -< (anyBackendConfig, backendInvalidationMap)
backendInfos <- resolveBackendCache -< (backendInvalidationMap, backendConfigs')
returnA -< backendInfo <> backendInfos
tryGetSourceConfig ::
forall b arr m.
( ArrowChoice arr,
Inc.ArrowCache m arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
MonadIO m,
MonadResolveSource m,
HasHttpManagerM m,
BackendMetadata b
) =>
( Inc.Dependency (HashMap SourceName Inc.InvalidationKey),
SourceName,
SourceConnConfiguration b,
BackendSourceKind b,
BackendInfo b
)
`arr` Maybe (SourceConfig b)
tryGetSourceConfig = Inc.cache proc (invalidationKeys, sourceName, sourceConfig, backendKind, backendInfo) -> do
let metadataObj = MetadataObject (MOSource sourceName) $ toJSON sourceName
httpMgr <- bindA -< askHttpManager
Inc.dependOn -< Inc.selectKeyD sourceName invalidationKeys
(|
withRecordInconsistency
( liftEitherA <<< bindA -< resolveSourceConfig @b logger sourceName sourceConfig backendKind backendInfo env httpMgr
)
|) metadataObj
tryResolveSource ::
forall b arr m.
( ArrowChoice arr,
Inc.ArrowCache m arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
MonadIO m,
MonadBaseControl IO m,
MonadResolveSource m,
HasHttpManagerM m,
BackendMetadata b
) =>
( Inc.Dependency (HashMap SourceName Inc.InvalidationKey),
BackendInfoAndSourceMetadata b
)
`arr` Maybe (ResolvedSource b)
tryResolveSource = Inc.cache proc (invalidationKeys, BackendInfoAndSourceMetadata {..}) -> do
let sourceName = _smName _bcasmSourceMetadata
metadataObj = MetadataObject (MOSource sourceName) $ toJSON sourceName
maybeSourceConfig <- tryGetSourceConfig @b -< (invalidationKeys, sourceName, _smConfiguration _bcasmSourceMetadata, _smKind _bcasmSourceMetadata, _bcasmBackendInfo)
case maybeSourceConfig of
Nothing -> returnA -< Nothing
Just sourceConfig ->
(|
withRecordInconsistency
( liftEitherA <<< bindA
-< do
resSource <- resolveDatabaseMetadata _bcasmSourceMetadata sourceConfig (getSourceTypeCustomization $ _smCustomization _bcasmSourceMetadata)
for_ resSource $ liftIO . unLogger logger
pure resSource
)
|) metadataObj
-- impl notes (swann):
--
-- as our cache invalidation key, we use the fact of the availability of event triggers
-- present, rerunning catalog init when this changes. i.e we invalidate the cache and
-- rebuild it with the catalog only when there is at least one event trigger present.
-- This is correct, because we only care about the transition from zero event triggers
-- to nonzero (not necessarily one, as Anon has observed, because replace_metadata can
-- add multiple event triggers in one go)
--
-- a future optimisation would be to cache, on a per-source basis, whether or not
-- the event catalog itself exists, and to then trigger catalog init when an event
-- trigger is created _but only if_ this cached information says the event catalog
-- doesn't already exist.
initCatalogIfNeeded ::
forall b arr m.
( ArrowChoice arr,
Inc.ArrowCache m arr,
MonadIO m,
BackendMetadata b,
HasServerConfigCtx m,
MonadError QErr m,
MonadBaseControl IO m
) =>
(Proxy b, Bool, SourceConfig b) `arr` (RecreateEventTriggers, SourceCatalogMigrationState)
initCatalogIfNeeded = Inc.cache proc (Proxy, atleastOneTrigger, sourceConfig) -> do
bindA
-< do
if atleastOneTrigger
then do
maintenanceMode <- _sccMaintenanceMode <$> askServerConfigCtx
eventingMode <- _sccEventingMode <$> askServerConfigCtx
readOnlyMode <- _sccReadOnlyMode <$> askServerConfigCtx
if
-- when safe mode is enabled, don't perform any migrations
| readOnlyMode == ReadOnlyModeEnabled -> pure (RETDoNothing, SCMSMigrationOnHold "read-only mode enabled")
-- when eventing mode is disabled, don't perform any migrations
| eventingMode == EventingDisabled -> pure (RETDoNothing, SCMSMigrationOnHold "eventing mode disabled")
-- when maintenance mode is enabled, don't perform any migrations
| maintenanceMode == (MaintenanceModeEnabled ()) -> pure (RETDoNothing, SCMSMigrationOnHold "maintenance mode enabled")
| otherwise -> do
-- The `initCatalogForSource` action is retried here because
-- in cloud there will be multiple workers (graphql-engine instances)
-- trying to migrate the source catalog, when needed. This introduces
-- a race condition as both the workers try to migrate the source catalog
-- concurrently and when one of them succeeds the other ones will fail
-- and be in an inconsistent state. To avoid the inconsistency, we retry
-- migrating the catalog on error and in the retry `initCatalogForSource`
-- will see that the catalog is already migrated, so it won't attempt the
-- migration again
liftEither
=<< Retry.retrying
( Retry.constantDelay (fromIntegral $ diffTimeToMicroSeconds $ seconds $ Seconds 10)
<> Retry.limitRetries 3
)
(const $ return . isLeft)
(const $ runExceptT $ prepareCatalog @b sourceConfig)
else pure (RETDoNothing, SCMSUninitializedSource)
buildSource ::
forall b arr m.
( ArrowChoice arr,
ArrowKleisli m arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
HasServerConfigCtx m,
MonadError QErr m,
BackendMetadata b,
GetAggregationPredicatesDeps b
) =>
( HashMap SourceName (AB.AnyBackend PartiallyResolvedSource),
SourceMetadata b,
SourceConfig b,
HashMap (TableName b) (TableCoreInfoG b (ColumnInfo b) (ColumnInfo b)),
HashMap (TableName b) (EventTriggerInfoMap b),
DBTablesMetadata b,
DBFunctionsMetadata b,
scaffolding for remote-schemas module The main aim of the PR is: 1. To set up a module structure for 'remote-schemas' package. 2. Move parts by the remote schema codebase into the new module structure to validate it. ## Notes to the reviewer Why a PR with large-ish diff? 1. We've been making progress on the MM project but we don't yet know long it is going to take us to get to the first milestone. To understand this better, we need to figure out the unknowns as soon as possible. Hence I've taken a stab at the first two items in the [end-state](https://gist.github.com/0x777/ca2bdc4284d21c3eec153b51dea255c9) document to figure out the unknowns. Unsurprisingly, there are a bunch of issues that we haven't discussed earlier. These are documented in the 'open questions' section. 1. The diff is large but that is only code moved around and I've added a section that documents how things are moved. In addition, there are fair number of PR comments to help with the review process. ## Changes in the PR ### Module structure Sets up the module structure as follows: ``` Hasura/ RemoteSchema/ Metadata/ Types.hs SchemaCache/ Types.hs Permission.hs RemoteRelationship.hs Build.hs MetadataAPI/ Types.hs Execute.hs ``` ### 1. Types representing metadata are moved Types that capture metadata information (currently scattered across several RQL modules) are moved into `Hasura.RemoteSchema.Metadata.Types`. - This new module only depends on very 'core' modules such as `Hasura.Session` for the notion of roles and `Hasura.Incremental` for `Cacheable` typeclass. - The requirement on database modules is avoided by generalizing the remote schemas metadata to accept an arbitrary 'r' for a remote relationship definition. ### 2. SchemaCache related types and build logic have been moved Types that represent remote schemas information in SchemaCache are moved into `Hasura.RemoteSchema.SchemaCache.Types`. Similar to `H.RS.Metadata.Types`, this module depends on 'core' modules except for `Hasura.GraphQL.Parser.Variable`. It has something to do with remote relationships but I haven't spent time looking into it. The validation of 'remote relationships to remote schema' is also something that needs to be looked at. Rips out the logic that builds remote schema's SchemaCache information from the monolithic `buildSchemaCacheRule` and moves it into `Hasura.RemoteSchema.SchemaCache.Build`. Further, the `.SchemaCache.Permission` and `.SchemaCache.RemoteRelationship` have been created from existing modules that capture schema cache building logic for those two components. This was a fair amount of work. On main, currently remote schema's SchemaCache information is built in two phases - in the first phase, 'permissions' and 'remote relationships' are ignored and in the second phase they are filled in. While remote relationships can only be resolved after partially resolving sources and other remote schemas, the same isn't true for permissions. Further, most of the work that is done to resolve remote relationships can be moved to the first phase so that the second phase can be a very simple traversal. This is the approach that was taken - resolve permissions and as much as remote relationships information in the first phase. ### 3. Metadata APIs related types and build logic have been moved The types that represent remote schema related metadata APIs and the execution logic have been moved to `Hasura.RemoteSchema.MetadataAPI.Types` and `.Execute` modules respectively. ## Open questions: 1. `Hasura.RemoteSchema.Metadata.Types` is so called because I was hoping that all of the metadata related APIs of remote schema can be brought in at `Hasura.RemoteSchema.Metadata.API`. However, as metadata APIs depended on functions from `SchemaCache` module (see [1](https://github.com/hasura/graphql-engine-mono/blob/ceba6d62264603ee5d279814677b29bcc43ecaea/server/src-lib/Hasura/RQL/DDL/RemoteSchema.hs#L55) and [2](https://github.com/hasura/graphql-engine-mono/blob/ceba6d62264603ee5d279814677b29bcc43ecaea/server/src-lib/Hasura/RQL/DDL/RemoteSchema.hs#L91), it made more sense to create a separate top-level module for `MetadataAPI`s. Maybe we can just have `Hasura.RemoteSchema.Metadata` and get rid of the extra nesting or have `Hasura.RemoteSchema.Metadata.{Core,Permission,RemoteRelationship}` if we want to break them down further. 1. `buildRemoteSchemas` in `H.RS.SchemaCache.Build` has the following type: ```haskell buildRemoteSchemas :: ( ArrowChoice arr, Inc.ArrowDistribute arr, ArrowWriter (Seq CollectedInfo) arr, Inc.ArrowCache m arr, MonadIO m, HasHttpManagerM m, Inc.Cacheable remoteRelationshipDefinition, ToJSON remoteRelationshipDefinition, MonadError QErr m ) => Env.Environment -> ( (Inc.Dependency (HashMap RemoteSchemaName Inc.InvalidationKey), OrderedRoles), [RemoteSchemaMetadataG remoteRelationshipDefinition] ) `arr` HashMap RemoteSchemaName (PartiallyResolvedRemoteSchemaCtxG remoteRelationshipDefinition, MetadataObject) ``` Note the dependence on `CollectedInfo` which is defined as ```haskell data CollectedInfo = CIInconsistency InconsistentMetadata | CIDependency MetadataObject -- ^ for error reporting on missing dependencies SchemaObjId SchemaDependency deriving (Eq) ``` this pretty much means that remote schemas is dependent on types from databases, actions, .... How do we fix this? Maybe introduce a typeclass such as `ArrowCollectRemoteSchemaDependencies` which is defined in `Hasura.RemoteSchema` and then implemented in graphql-engine? 1. The dependency on `buildSchemaCacheFor` in `.MetadataAPI.Execute` which has the following signature: ```haskell buildSchemaCacheFor :: (QErrM m, CacheRWM m, MetadataM m) => MetadataObjId -> MetadataModifier -> ``` This can be easily resolved if we restrict what the metadata APIs are allowed to do. Currently, they operate in an unfettered access to modify SchemaCache (the `CacheRWM` constraint): ```haskell runAddRemoteSchema :: ( QErrM m, CacheRWM m, MonadIO m, HasHttpManagerM m, MetadataM m, Tracing.MonadTrace m ) => Env.Environment -> AddRemoteSchemaQuery -> m EncJSON ``` This should instead be changed to restrict remote schema APIs to only modify remote schema metadata (but has access to the remote schemas part of the schema cache), this dependency is completely removed. ```haskell runAddRemoteSchema :: ( QErrM m, MonadIO m, HasHttpManagerM m, MonadReader RemoteSchemasSchemaCache m, MonadState RemoteSchemaMetadata m, Tracing.MonadTrace m ) => Env.Environment -> AddRemoteSchemaQuery -> m RemoteSchemeMetadataObjId ``` The idea is that the core graphql-engine would call these functions and then call `buildSchemaCacheFor`. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6291 GitOrigin-RevId: 51357148c6404afe70219afa71bd1d59bdf4ffc6
2022-10-21 06:13:07 +03:00
PartiallyResolvedRemoteSchemaMap,
OrderedRoles
)
`arr` (SourceInfo b)
buildSource = proc (allSources, sourceMetadata, sourceConfig, tablesRawInfo, eventTriggerInfoMaps, _dbTables, dbFunctions, remoteSchemaMap, orderedRoles) -> do
let SourceMetadata sourceName _backendKind tables functions _ queryTagsConfig sourceCustomization _healthCheckConfig = sourceMetadata
tablesMetadata = OMap.elems tables
(_, nonColumnInputs, permissions) = unzip3 $ map mkTableInputs tablesMetadata
alignTableMap :: HashMap (TableName b) a -> HashMap (TableName b) c -> HashMap (TableName b) (a, c)
alignTableMap = M.intersectionWith (,)
-- relationships and computed fields
let nonColumnsByTable = mapFromL _nctiTable nonColumnInputs
tableCoreInfos :: HashMap (TableName b) (TableCoreInfo b) <-
interpretWriter
-< for (tablesRawInfo `alignTableMap` nonColumnsByTable) \(tableRawInfo, nonColumnInput) -> do
let columns = _tciFieldInfoMap tableRawInfo
allFields :: FieldInfoMap (FieldInfo b) <- addNonColumnFields allSources sourceName tablesRawInfo columns remoteSchemaMap dbFunctions nonColumnInput
pure $ tableRawInfo {_tciFieldInfoMap = allFields}
-- permissions
result <-
interpretWriter
-< runExceptT $
for
(tableCoreInfos `alignTableMap` mapFromL _tpiTable permissions `alignTableMap` eventTriggerInfoMaps)
\((tableCoreInfo, permissionInputs), eventTriggerInfos) -> do
let tableFields = _tciFieldInfoMap tableCoreInfo
permissionInfos <-
buildTablePermissions
sourceName
tableCoreInfos
tableFields
permissionInputs
orderedRoles
pure $ TableInfo tableCoreInfo permissionInfos eventTriggerInfos (mkAdminRolePermInfo tableCoreInfo)
-- Generate a non-recoverable error when inherited roles were not ordered in a way that allows for building permissions to succeed
tableCache <- bindA -< liftEither result
Resolve source customization at schema cache building time. ### Description This PR attempts to fix several issues with source customization as it relates to remote relationships. There were several issues regarding casing: at the relationship border, we didn't properly set the target source's case, we didn't have access to the list of supported features to decide whether the feature was allowed or not, and we didn't have access to the global default. However, all of that information is available when we build the schema cache, as we do resolve the case of some elements such as function names: we can therefore resolve source information at the same time, and simplify both the root of the schema and the remote relationship border. To do this, this PR introduces a new type, `ResolvedSourceCustomization`, to be used in the Schema Cache, as opposed to the metadata's `SourceCustomization`, following a pattern established by a lot of other types. ### Remaining work and open questions One major point of confusion: it seems to me that we didn't set the case at all across remote relationships, which would suggest we would use the case of the LHS source across the subset of the RHS one that is accessible through the remote relationship, which would in turn "corrupt" the parser cache and might result in the wrong case being used for that source later on. Is that assesment correct, and was I right to fix it? Another one is that we seem not to be using the local case of the RHS to name the field in an object relationship; unless I'm mistaken we only use it for array relationships? Is that intentional? This PR is also missing tests that would show-case the difference, and a changelog entry. To my knowledge, all the tests of this feature are in the python test suite; this could be the opportunity to move them to the hspec suite, but this might be a considerable amount of work? PR-URL: https://github.com/hasura/graphql-engine-mono/pull/5619 GitOrigin-RevId: 51a81b713a74575e82d9f96b51633f158ce3a47b
2022-09-12 19:05:40 +03:00
-- not forcing the evaluation here results in a measurable negative impact
-- on memory residency as measured by our benchmark
!defaultNC <- bindA -< _sccDefaultNamingConvention <$> askServerConfigCtx
!isNamingConventionEnabled <- bindA -< ((EFNamingConventions `elem`) . _sccExperimentalFeatures) <$> askServerConfigCtx
Resolve source customization at schema cache building time. ### Description This PR attempts to fix several issues with source customization as it relates to remote relationships. There were several issues regarding casing: at the relationship border, we didn't properly set the target source's case, we didn't have access to the list of supported features to decide whether the feature was allowed or not, and we didn't have access to the global default. However, all of that information is available when we build the schema cache, as we do resolve the case of some elements such as function names: we can therefore resolve source information at the same time, and simplify both the root of the schema and the remote relationship border. To do this, this PR introduces a new type, `ResolvedSourceCustomization`, to be used in the Schema Cache, as opposed to the metadata's `SourceCustomization`, following a pattern established by a lot of other types. ### Remaining work and open questions One major point of confusion: it seems to me that we didn't set the case at all across remote relationships, which would suggest we would use the case of the LHS source across the subset of the RHS one that is accessible through the remote relationship, which would in turn "corrupt" the parser cache and might result in the wrong case being used for that source later on. Is that assesment correct, and was I right to fix it? Another one is that we seem not to be using the local case of the RHS to name the field in an object relationship; unless I'm mistaken we only use it for array relationships? Is that intentional? This PR is also missing tests that would show-case the difference, and a changelog entry. To my knowledge, all the tests of this feature are in the python test suite; this could be the opportunity to move them to the hspec suite, but this might be a considerable amount of work? PR-URL: https://github.com/hasura/graphql-engine-mono/pull/5619 GitOrigin-RevId: 51a81b713a74575e82d9f96b51633f158ce3a47b
2022-09-12 19:05:40 +03:00
!namingConv <-
bindA
-<
if isNamingConventionEnabled
then getNamingCase sourceCustomization (namingConventionSupport @b) defaultNC
else pure HasuraCase
let resolvedCustomization = mkResolvedSourceCustomization sourceCustomization namingConv
-- sql functions
functionCacheMaybes <-
interpretWriter
-< for
(OMap.elems functions)
\case
FunctionMetadata qf config functionPermissions comment -> do
let systemDefined = SystemDefined False
definition = TrackFunction @b qf
metadataObject =
MetadataObject
( MOSourceObjId sourceName $
AB.mkAnyBackend $
SMOFunction @b qf
)
(toJSON definition)
schemaObject =
SOSourceObj sourceName $
AB.mkAnyBackend $
SOIFunction @b qf
addFunctionContext e = "in function " <> qf <<> ": " <> e
funcDefs = fromMaybe [] $ M.lookup qf dbFunctions
metadataPermissions = mapFromL _fpmRole functionPermissions
permissionsMap = mkBooleanPermissionMap FunctionPermissionInfo metadataPermissions orderedRoles
withRecordInconsistencyM metadataObject $ modifyErr addFunctionContext do
rawfunctionInfo <- handleMultipleFunctions @b qf funcDefs
(functionInfo, dep) <- buildFunctionInfo sourceName qf systemDefined config permissionsMap rawfunctionInfo comment namingConv
recordDependenciesM metadataObject schemaObject [dep]
pure functionInfo
let functionCache = mapFromL _fiSQLName $ catMaybes functionCacheMaybes
returnA -< SourceInfo sourceName tableCache functionCache sourceConfig queryTagsConfig resolvedCustomization
buildAndCollectInfo ::
forall arr m.
( ArrowChoice arr,
Inc.ArrowDistribute arr,
Inc.ArrowCache m arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
MonadIO m,
MonadError QErr m,
MonadReader BuildReason m,
MonadBaseControl IO m,
HasHttpManagerM m,
HasServerConfigCtx m,
MonadResolveSource m
) =>
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
(Inc.Dependency Metadata, Inc.Dependency InvalidationKeys) `arr` BuildOutputs
buildAndCollectInfo = proc (metadataDep, invalidationKeys) -> do
sources <- Inc.dependOn -< Inc.selectD #_metaSources metadataDep
remoteSchemas <- Inc.dependOn -< Inc.selectD #_metaRemoteSchemas metadataDep
customTypes <- Inc.dependOn -< Inc.selectD #_metaCustomTypes metadataDep
actions <- Inc.dependOn -< Inc.selectD #_metaActions metadataDep
inheritedRoles <- Inc.dependOn -< Inc.selectD #_metaInheritedRoles metadataDep
backendConfigs <- Inc.dependOn -< Inc.selectD #_metaBackendConfigs metadataDep
let actionRoles = map _apmRole . _amPermissions =<< OMap.elems actions
[Preview] Inherited roles for postgres read queries fixes #3868 docker image - `hasura/graphql-engine:inherited-roles-preview-48b73a2de` Note: To be able to use the inherited roles feature, the graphql-engine should be started with the env variable `HASURA_GRAPHQL_EXPERIMENTAL_FEATURES` set to `inherited_roles`. Introduction ------------ This PR implements the idea of multiple roles as presented in this [paper](https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf). The multiple roles feature in this PR can be used via inherited roles. An inherited role is a role which can be created by combining multiple singular roles. For example, if there are two roles `author` and `editor` configured in the graphql-engine, then we can create a inherited role with the name of `combined_author_editor` role which will combine the select permissions of the `author` and `editor` roles and then make GraphQL queries using the `combined_author_editor`. How are select permissions of different roles are combined? ------------------------------------------------------------ A select permission includes 5 things: 1. Columns accessible to the role 2. Row selection filter 3. Limit 4. Allow aggregation 5. Scalar computed fields accessible to the role Suppose there are two roles, `role1` gives access to the `address` column with row filter `P1` and `role2` gives access to both the `address` and the `phone` column with row filter `P2` and we create a new role `combined_roles` which combines `role1` and `role2`. Let's say the following GraphQL query is queried with the `combined_roles` role. ```graphql query { employees { address phone } } ``` This will translate to the following SQL query: ```sql select (case when (P1 or P2) then address else null end) as address, (case when P2 then phone else null end) as phone from employee where (P1 or P2) ``` The other parameters of the select permission will be combined in the following manner: 1. Limit - Minimum of the limits will be the limit of the inherited role 2. Allow aggregations - If any of the role allows aggregation, then the inherited role will allow aggregation 3. Scalar computed fields - same as table column fields, as in the above example APIs for inherited roles: ---------------------- 1. `add_inherited_role` `add_inherited_role` is the [metadata API](https://hasura.io/docs/1.0/graphql/core/api-reference/index.html#schema-metadata-api) to create a new inherited role. It accepts two arguments `role_name`: the name of the inherited role to be added (String) `role_set`: list of roles that need to be combined (Array of Strings) Example: ```json { "type": "add_inherited_role", "args": { "role_name":"combined_user", "role_set":[ "user", "user1" ] } } ``` After adding the inherited role, the inherited role can be used like single roles like earlier Note: An inherited role can only be created with non-inherited/singular roles. 2. `drop_inherited_role` The `drop_inherited_role` API accepts the name of the inherited role and drops it from the metadata. It accepts a single argument: `role_name`: name of the inherited role to be dropped Example: ```json { "type": "drop_inherited_role", "args": { "role_name":"combined_user" } } ``` Metadata --------- The derived roles metadata will be included under the `experimental_features` key while exporting the metadata. ```json { "experimental_features": { "derived_roles": [ { "role_name": "manager_is_employee_too", "role_set": [ "employee", "manager" ] } ] } } ``` Scope ------ Only postgres queries and subscriptions are supported in this PR. Important points: ----------------- 1. All columns exposed to an inherited role will be marked as `nullable`, this is done so that cell value nullification can be done. TODOs ------- - [ ] Tests - [ ] Test a GraphQL query running with a inherited role without enabling inherited roles in experimental features - [] Tests for aggregate queries, limit, computed fields, functions, subscriptions (?) - [ ] Introspection test with a inherited role (nullability changes in a inherited role) - [ ] Docs - [ ] Changelog Co-authored-by: Vamshi Surabhi <6562944+0x777@users.noreply.github.com> GitOrigin-RevId: 3b8ee1e11f5ceca80fe294f8c074d42fbccfec63
2021-03-08 14:14:13 +03:00
remoteSchemaRoles = map _rspmRole . _rsmPermissions =<< OMap.elems remoteSchemas
sourceRoles =
HS.fromList $
concat $
OMap.elems sources >>= \(BackendSourceMetadata e) ->
AB.dispatchAnyBackend @Backend e \(SourceMetadata _ _ tables _functions _ _ _ _) -> do
table <- OMap.elems tables
pure $
OMap.keys (_tmInsertPermissions table)
<> OMap.keys (_tmSelectPermissions table)
<> OMap.keys (_tmUpdatePermissions table)
<> OMap.keys (_tmDeletePermissions table)
inheritedRoleNames = OMap.keys inheritedRoles
allRoleNames = sourceRoles <> HS.fromList (remoteSchemaRoles <> actionRoles <> inheritedRoleNames)
[Preview] Inherited roles for postgres read queries fixes #3868 docker image - `hasura/graphql-engine:inherited-roles-preview-48b73a2de` Note: To be able to use the inherited roles feature, the graphql-engine should be started with the env variable `HASURA_GRAPHQL_EXPERIMENTAL_FEATURES` set to `inherited_roles`. Introduction ------------ This PR implements the idea of multiple roles as presented in this [paper](https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf). The multiple roles feature in this PR can be used via inherited roles. An inherited role is a role which can be created by combining multiple singular roles. For example, if there are two roles `author` and `editor` configured in the graphql-engine, then we can create a inherited role with the name of `combined_author_editor` role which will combine the select permissions of the `author` and `editor` roles and then make GraphQL queries using the `combined_author_editor`. How are select permissions of different roles are combined? ------------------------------------------------------------ A select permission includes 5 things: 1. Columns accessible to the role 2. Row selection filter 3. Limit 4. Allow aggregation 5. Scalar computed fields accessible to the role Suppose there are two roles, `role1` gives access to the `address` column with row filter `P1` and `role2` gives access to both the `address` and the `phone` column with row filter `P2` and we create a new role `combined_roles` which combines `role1` and `role2`. Let's say the following GraphQL query is queried with the `combined_roles` role. ```graphql query { employees { address phone } } ``` This will translate to the following SQL query: ```sql select (case when (P1 or P2) then address else null end) as address, (case when P2 then phone else null end) as phone from employee where (P1 or P2) ``` The other parameters of the select permission will be combined in the following manner: 1. Limit - Minimum of the limits will be the limit of the inherited role 2. Allow aggregations - If any of the role allows aggregation, then the inherited role will allow aggregation 3. Scalar computed fields - same as table column fields, as in the above example APIs for inherited roles: ---------------------- 1. `add_inherited_role` `add_inherited_role` is the [metadata API](https://hasura.io/docs/1.0/graphql/core/api-reference/index.html#schema-metadata-api) to create a new inherited role. It accepts two arguments `role_name`: the name of the inherited role to be added (String) `role_set`: list of roles that need to be combined (Array of Strings) Example: ```json { "type": "add_inherited_role", "args": { "role_name":"combined_user", "role_set":[ "user", "user1" ] } } ``` After adding the inherited role, the inherited role can be used like single roles like earlier Note: An inherited role can only be created with non-inherited/singular roles. 2. `drop_inherited_role` The `drop_inherited_role` API accepts the name of the inherited role and drops it from the metadata. It accepts a single argument: `role_name`: name of the inherited role to be dropped Example: ```json { "type": "drop_inherited_role", "args": { "role_name":"combined_user" } } ``` Metadata --------- The derived roles metadata will be included under the `experimental_features` key while exporting the metadata. ```json { "experimental_features": { "derived_roles": [ { "role_name": "manager_is_employee_too", "role_set": [ "employee", "manager" ] } ] } } ``` Scope ------ Only postgres queries and subscriptions are supported in this PR. Important points: ----------------- 1. All columns exposed to an inherited role will be marked as `nullable`, this is done so that cell value nullification can be done. TODOs ------- - [ ] Tests - [ ] Test a GraphQL query running with a inherited role without enabling inherited roles in experimental features - [] Tests for aggregate queries, limit, computed fields, functions, subscriptions (?) - [ ] Introspection test with a inherited role (nullability changes in a inherited role) - [ ] Docs - [ ] Changelog Co-authored-by: Vamshi Surabhi <6562944+0x777@users.noreply.github.com> GitOrigin-RevId: 3b8ee1e11f5ceca80fe294f8c074d42fbccfec63
2021-03-08 14:14:13 +03:00
-- roles which have some kind of permission (action/remote schema/table/function) set in the metadata
let metadataRoles = mapFromL _rRoleName $ (`Role` ParentRoles mempty) <$> toList allRoleNames
[Preview] Inherited roles for postgres read queries fixes #3868 docker image - `hasura/graphql-engine:inherited-roles-preview-48b73a2de` Note: To be able to use the inherited roles feature, the graphql-engine should be started with the env variable `HASURA_GRAPHQL_EXPERIMENTAL_FEATURES` set to `inherited_roles`. Introduction ------------ This PR implements the idea of multiple roles as presented in this [paper](https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf). The multiple roles feature in this PR can be used via inherited roles. An inherited role is a role which can be created by combining multiple singular roles. For example, if there are two roles `author` and `editor` configured in the graphql-engine, then we can create a inherited role with the name of `combined_author_editor` role which will combine the select permissions of the `author` and `editor` roles and then make GraphQL queries using the `combined_author_editor`. How are select permissions of different roles are combined? ------------------------------------------------------------ A select permission includes 5 things: 1. Columns accessible to the role 2. Row selection filter 3. Limit 4. Allow aggregation 5. Scalar computed fields accessible to the role Suppose there are two roles, `role1` gives access to the `address` column with row filter `P1` and `role2` gives access to both the `address` and the `phone` column with row filter `P2` and we create a new role `combined_roles` which combines `role1` and `role2`. Let's say the following GraphQL query is queried with the `combined_roles` role. ```graphql query { employees { address phone } } ``` This will translate to the following SQL query: ```sql select (case when (P1 or P2) then address else null end) as address, (case when P2 then phone else null end) as phone from employee where (P1 or P2) ``` The other parameters of the select permission will be combined in the following manner: 1. Limit - Minimum of the limits will be the limit of the inherited role 2. Allow aggregations - If any of the role allows aggregation, then the inherited role will allow aggregation 3. Scalar computed fields - same as table column fields, as in the above example APIs for inherited roles: ---------------------- 1. `add_inherited_role` `add_inherited_role` is the [metadata API](https://hasura.io/docs/1.0/graphql/core/api-reference/index.html#schema-metadata-api) to create a new inherited role. It accepts two arguments `role_name`: the name of the inherited role to be added (String) `role_set`: list of roles that need to be combined (Array of Strings) Example: ```json { "type": "add_inherited_role", "args": { "role_name":"combined_user", "role_set":[ "user", "user1" ] } } ``` After adding the inherited role, the inherited role can be used like single roles like earlier Note: An inherited role can only be created with non-inherited/singular roles. 2. `drop_inherited_role` The `drop_inherited_role` API accepts the name of the inherited role and drops it from the metadata. It accepts a single argument: `role_name`: name of the inherited role to be dropped Example: ```json { "type": "drop_inherited_role", "args": { "role_name":"combined_user" } } ``` Metadata --------- The derived roles metadata will be included under the `experimental_features` key while exporting the metadata. ```json { "experimental_features": { "derived_roles": [ { "role_name": "manager_is_employee_too", "role_set": [ "employee", "manager" ] } ] } } ``` Scope ------ Only postgres queries and subscriptions are supported in this PR. Important points: ----------------- 1. All columns exposed to an inherited role will be marked as `nullable`, this is done so that cell value nullification can be done. TODOs ------- - [ ] Tests - [ ] Test a GraphQL query running with a inherited role without enabling inherited roles in experimental features - [] Tests for aggregate queries, limit, computed fields, functions, subscriptions (?) - [ ] Introspection test with a inherited role (nullability changes in a inherited role) - [ ] Docs - [ ] Changelog Co-authored-by: Vamshi Surabhi <6562944+0x777@users.noreply.github.com> GitOrigin-RevId: 3b8ee1e11f5ceca80fe294f8c074d42fbccfec63
2021-03-08 14:14:13 +03:00
resolvedInheritedRoles <- interpretWriter -< buildInheritedRoles allRoleNames (OMap.elems inheritedRoles)
[Preview] Inherited roles for postgres read queries fixes #3868 docker image - `hasura/graphql-engine:inherited-roles-preview-48b73a2de` Note: To be able to use the inherited roles feature, the graphql-engine should be started with the env variable `HASURA_GRAPHQL_EXPERIMENTAL_FEATURES` set to `inherited_roles`. Introduction ------------ This PR implements the idea of multiple roles as presented in this [paper](https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf). The multiple roles feature in this PR can be used via inherited roles. An inherited role is a role which can be created by combining multiple singular roles. For example, if there are two roles `author` and `editor` configured in the graphql-engine, then we can create a inherited role with the name of `combined_author_editor` role which will combine the select permissions of the `author` and `editor` roles and then make GraphQL queries using the `combined_author_editor`. How are select permissions of different roles are combined? ------------------------------------------------------------ A select permission includes 5 things: 1. Columns accessible to the role 2. Row selection filter 3. Limit 4. Allow aggregation 5. Scalar computed fields accessible to the role Suppose there are two roles, `role1` gives access to the `address` column with row filter `P1` and `role2` gives access to both the `address` and the `phone` column with row filter `P2` and we create a new role `combined_roles` which combines `role1` and `role2`. Let's say the following GraphQL query is queried with the `combined_roles` role. ```graphql query { employees { address phone } } ``` This will translate to the following SQL query: ```sql select (case when (P1 or P2) then address else null end) as address, (case when P2 then phone else null end) as phone from employee where (P1 or P2) ``` The other parameters of the select permission will be combined in the following manner: 1. Limit - Minimum of the limits will be the limit of the inherited role 2. Allow aggregations - If any of the role allows aggregation, then the inherited role will allow aggregation 3. Scalar computed fields - same as table column fields, as in the above example APIs for inherited roles: ---------------------- 1. `add_inherited_role` `add_inherited_role` is the [metadata API](https://hasura.io/docs/1.0/graphql/core/api-reference/index.html#schema-metadata-api) to create a new inherited role. It accepts two arguments `role_name`: the name of the inherited role to be added (String) `role_set`: list of roles that need to be combined (Array of Strings) Example: ```json { "type": "add_inherited_role", "args": { "role_name":"combined_user", "role_set":[ "user", "user1" ] } } ``` After adding the inherited role, the inherited role can be used like single roles like earlier Note: An inherited role can only be created with non-inherited/singular roles. 2. `drop_inherited_role` The `drop_inherited_role` API accepts the name of the inherited role and drops it from the metadata. It accepts a single argument: `role_name`: name of the inherited role to be dropped Example: ```json { "type": "drop_inherited_role", "args": { "role_name":"combined_user" } } ``` Metadata --------- The derived roles metadata will be included under the `experimental_features` key while exporting the metadata. ```json { "experimental_features": { "derived_roles": [ { "role_name": "manager_is_employee_too", "role_set": [ "employee", "manager" ] } ] } } ``` Scope ------ Only postgres queries and subscriptions are supported in this PR. Important points: ----------------- 1. All columns exposed to an inherited role will be marked as `nullable`, this is done so that cell value nullification can be done. TODOs ------- - [ ] Tests - [ ] Test a GraphQL query running with a inherited role without enabling inherited roles in experimental features - [] Tests for aggregate queries, limit, computed fields, functions, subscriptions (?) - [ ] Introspection test with a inherited role (nullability changes in a inherited role) - [ ] Docs - [ ] Changelog Co-authored-by: Vamshi Surabhi <6562944+0x777@users.noreply.github.com> GitOrigin-RevId: 3b8ee1e11f5ceca80fe294f8c074d42fbccfec63
2021-03-08 14:14:13 +03:00
let allRoles = resolvedInheritedRoles `M.union` metadataRoles
[Preview] Inherited roles for postgres read queries fixes #3868 docker image - `hasura/graphql-engine:inherited-roles-preview-48b73a2de` Note: To be able to use the inherited roles feature, the graphql-engine should be started with the env variable `HASURA_GRAPHQL_EXPERIMENTAL_FEATURES` set to `inherited_roles`. Introduction ------------ This PR implements the idea of multiple roles as presented in this [paper](https://www.microsoft.com/en-us/research/wp-content/uploads/2016/02/FGALanguageICDE07.pdf). The multiple roles feature in this PR can be used via inherited roles. An inherited role is a role which can be created by combining multiple singular roles. For example, if there are two roles `author` and `editor` configured in the graphql-engine, then we can create a inherited role with the name of `combined_author_editor` role which will combine the select permissions of the `author` and `editor` roles and then make GraphQL queries using the `combined_author_editor`. How are select permissions of different roles are combined? ------------------------------------------------------------ A select permission includes 5 things: 1. Columns accessible to the role 2. Row selection filter 3. Limit 4. Allow aggregation 5. Scalar computed fields accessible to the role Suppose there are two roles, `role1` gives access to the `address` column with row filter `P1` and `role2` gives access to both the `address` and the `phone` column with row filter `P2` and we create a new role `combined_roles` which combines `role1` and `role2`. Let's say the following GraphQL query is queried with the `combined_roles` role. ```graphql query { employees { address phone } } ``` This will translate to the following SQL query: ```sql select (case when (P1 or P2) then address else null end) as address, (case when P2 then phone else null end) as phone from employee where (P1 or P2) ``` The other parameters of the select permission will be combined in the following manner: 1. Limit - Minimum of the limits will be the limit of the inherited role 2. Allow aggregations - If any of the role allows aggregation, then the inherited role will allow aggregation 3. Scalar computed fields - same as table column fields, as in the above example APIs for inherited roles: ---------------------- 1. `add_inherited_role` `add_inherited_role` is the [metadata API](https://hasura.io/docs/1.0/graphql/core/api-reference/index.html#schema-metadata-api) to create a new inherited role. It accepts two arguments `role_name`: the name of the inherited role to be added (String) `role_set`: list of roles that need to be combined (Array of Strings) Example: ```json { "type": "add_inherited_role", "args": { "role_name":"combined_user", "role_set":[ "user", "user1" ] } } ``` After adding the inherited role, the inherited role can be used like single roles like earlier Note: An inherited role can only be created with non-inherited/singular roles. 2. `drop_inherited_role` The `drop_inherited_role` API accepts the name of the inherited role and drops it from the metadata. It accepts a single argument: `role_name`: name of the inherited role to be dropped Example: ```json { "type": "drop_inherited_role", "args": { "role_name":"combined_user" } } ``` Metadata --------- The derived roles metadata will be included under the `experimental_features` key while exporting the metadata. ```json { "experimental_features": { "derived_roles": [ { "role_name": "manager_is_employee_too", "role_set": [ "employee", "manager" ] } ] } } ``` Scope ------ Only postgres queries and subscriptions are supported in this PR. Important points: ----------------- 1. All columns exposed to an inherited role will be marked as `nullable`, this is done so that cell value nullification can be done. TODOs ------- - [ ] Tests - [ ] Test a GraphQL query running with a inherited role without enabling inherited roles in experimental features - [] Tests for aggregate queries, limit, computed fields, functions, subscriptions (?) - [ ] Introspection test with a inherited role (nullability changes in a inherited role) - [ ] Docs - [ ] Changelog Co-authored-by: Vamshi Surabhi <6562944+0x777@users.noreply.github.com> GitOrigin-RevId: 3b8ee1e11f5ceca80fe294f8c074d42fbccfec63
2021-03-08 14:14:13 +03:00
orderedRoles <- bindA -< orderRoles $ M.elems allRoles
-- remote schemas
let remoteSchemaInvalidationKeys = Inc.selectD #_ikRemoteSchemas invalidationKeys
scaffolding for remote-schemas module The main aim of the PR is: 1. To set up a module structure for 'remote-schemas' package. 2. Move parts by the remote schema codebase into the new module structure to validate it. ## Notes to the reviewer Why a PR with large-ish diff? 1. We've been making progress on the MM project but we don't yet know long it is going to take us to get to the first milestone. To understand this better, we need to figure out the unknowns as soon as possible. Hence I've taken a stab at the first two items in the [end-state](https://gist.github.com/0x777/ca2bdc4284d21c3eec153b51dea255c9) document to figure out the unknowns. Unsurprisingly, there are a bunch of issues that we haven't discussed earlier. These are documented in the 'open questions' section. 1. The diff is large but that is only code moved around and I've added a section that documents how things are moved. In addition, there are fair number of PR comments to help with the review process. ## Changes in the PR ### Module structure Sets up the module structure as follows: ``` Hasura/ RemoteSchema/ Metadata/ Types.hs SchemaCache/ Types.hs Permission.hs RemoteRelationship.hs Build.hs MetadataAPI/ Types.hs Execute.hs ``` ### 1. Types representing metadata are moved Types that capture metadata information (currently scattered across several RQL modules) are moved into `Hasura.RemoteSchema.Metadata.Types`. - This new module only depends on very 'core' modules such as `Hasura.Session` for the notion of roles and `Hasura.Incremental` for `Cacheable` typeclass. - The requirement on database modules is avoided by generalizing the remote schemas metadata to accept an arbitrary 'r' for a remote relationship definition. ### 2. SchemaCache related types and build logic have been moved Types that represent remote schemas information in SchemaCache are moved into `Hasura.RemoteSchema.SchemaCache.Types`. Similar to `H.RS.Metadata.Types`, this module depends on 'core' modules except for `Hasura.GraphQL.Parser.Variable`. It has something to do with remote relationships but I haven't spent time looking into it. The validation of 'remote relationships to remote schema' is also something that needs to be looked at. Rips out the logic that builds remote schema's SchemaCache information from the monolithic `buildSchemaCacheRule` and moves it into `Hasura.RemoteSchema.SchemaCache.Build`. Further, the `.SchemaCache.Permission` and `.SchemaCache.RemoteRelationship` have been created from existing modules that capture schema cache building logic for those two components. This was a fair amount of work. On main, currently remote schema's SchemaCache information is built in two phases - in the first phase, 'permissions' and 'remote relationships' are ignored and in the second phase they are filled in. While remote relationships can only be resolved after partially resolving sources and other remote schemas, the same isn't true for permissions. Further, most of the work that is done to resolve remote relationships can be moved to the first phase so that the second phase can be a very simple traversal. This is the approach that was taken - resolve permissions and as much as remote relationships information in the first phase. ### 3. Metadata APIs related types and build logic have been moved The types that represent remote schema related metadata APIs and the execution logic have been moved to `Hasura.RemoteSchema.MetadataAPI.Types` and `.Execute` modules respectively. ## Open questions: 1. `Hasura.RemoteSchema.Metadata.Types` is so called because I was hoping that all of the metadata related APIs of remote schema can be brought in at `Hasura.RemoteSchema.Metadata.API`. However, as metadata APIs depended on functions from `SchemaCache` module (see [1](https://github.com/hasura/graphql-engine-mono/blob/ceba6d62264603ee5d279814677b29bcc43ecaea/server/src-lib/Hasura/RQL/DDL/RemoteSchema.hs#L55) and [2](https://github.com/hasura/graphql-engine-mono/blob/ceba6d62264603ee5d279814677b29bcc43ecaea/server/src-lib/Hasura/RQL/DDL/RemoteSchema.hs#L91), it made more sense to create a separate top-level module for `MetadataAPI`s. Maybe we can just have `Hasura.RemoteSchema.Metadata` and get rid of the extra nesting or have `Hasura.RemoteSchema.Metadata.{Core,Permission,RemoteRelationship}` if we want to break them down further. 1. `buildRemoteSchemas` in `H.RS.SchemaCache.Build` has the following type: ```haskell buildRemoteSchemas :: ( ArrowChoice arr, Inc.ArrowDistribute arr, ArrowWriter (Seq CollectedInfo) arr, Inc.ArrowCache m arr, MonadIO m, HasHttpManagerM m, Inc.Cacheable remoteRelationshipDefinition, ToJSON remoteRelationshipDefinition, MonadError QErr m ) => Env.Environment -> ( (Inc.Dependency (HashMap RemoteSchemaName Inc.InvalidationKey), OrderedRoles), [RemoteSchemaMetadataG remoteRelationshipDefinition] ) `arr` HashMap RemoteSchemaName (PartiallyResolvedRemoteSchemaCtxG remoteRelationshipDefinition, MetadataObject) ``` Note the dependence on `CollectedInfo` which is defined as ```haskell data CollectedInfo = CIInconsistency InconsistentMetadata | CIDependency MetadataObject -- ^ for error reporting on missing dependencies SchemaObjId SchemaDependency deriving (Eq) ``` this pretty much means that remote schemas is dependent on types from databases, actions, .... How do we fix this? Maybe introduce a typeclass such as `ArrowCollectRemoteSchemaDependencies` which is defined in `Hasura.RemoteSchema` and then implemented in graphql-engine? 1. The dependency on `buildSchemaCacheFor` in `.MetadataAPI.Execute` which has the following signature: ```haskell buildSchemaCacheFor :: (QErrM m, CacheRWM m, MetadataM m) => MetadataObjId -> MetadataModifier -> ``` This can be easily resolved if we restrict what the metadata APIs are allowed to do. Currently, they operate in an unfettered access to modify SchemaCache (the `CacheRWM` constraint): ```haskell runAddRemoteSchema :: ( QErrM m, CacheRWM m, MonadIO m, HasHttpManagerM m, MetadataM m, Tracing.MonadTrace m ) => Env.Environment -> AddRemoteSchemaQuery -> m EncJSON ``` This should instead be changed to restrict remote schema APIs to only modify remote schema metadata (but has access to the remote schemas part of the schema cache), this dependency is completely removed. ```haskell runAddRemoteSchema :: ( QErrM m, MonadIO m, HasHttpManagerM m, MonadReader RemoteSchemasSchemaCache m, MonadState RemoteSchemaMetadata m, Tracing.MonadTrace m ) => Env.Environment -> AddRemoteSchemaQuery -> m RemoteSchemeMetadataObjId ``` The idea is that the core graphql-engine would call these functions and then call `buildSchemaCacheFor`. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6291 GitOrigin-RevId: 51357148c6404afe70219afa71bd1d59bdf4ffc6
2022-10-21 06:13:07 +03:00
remoteSchemaMap <- buildRemoteSchemas env -< ((remoteSchemaInvalidationKeys, orderedRoles), OMap.elems remoteSchemas)
let remoteSchemaCtxMap = M.map fst remoteSchemaMap
!defaultNC <- bindA -< _sccDefaultNamingConvention <$> askServerConfigCtx
!isNamingConventionEnabled <- bindA -< ((EFNamingConventions `elem`) . _sccExperimentalFeatures) <$> askServerConfigCtx
let backendInvalidationKeys = Inc.selectD #_ikBackends invalidationKeys
backendCache <- resolveBackendCache -< (backendInvalidationKeys, BackendMap.elems backendConfigs)
let backendInfoAndSourceMetadata = joinBackendInfosToSources backendCache sources
-- sources are build in two steps
-- first we resolve them, and build the table cache
partiallyResolvedSourcesMaybes <-
(|
Inc.keyed
( \_ exists ->
AB.dispatchAnyBackendArrow @BackendMetadata @BackendEventTrigger
( proc (backendInfoAndSourceMetadata, (invalidationKeys, defaultNC, isNamingConventionEnabled)) -> do
let sourceMetadata = _bcasmSourceMetadata backendInfoAndSourceMetadata
sourceName = _smName sourceMetadata
sourceInvalidationsKeys = Inc.selectD #_ikSources invalidationKeys
maybeResolvedSource <- tryResolveSource -< (sourceInvalidationsKeys, backendInfoAndSourceMetadata)
case maybeResolvedSource of
Nothing -> returnA -< Nothing
Just (source :: ResolvedSource b) -> do
let metadataInvalidationKey = Inc.selectD #_ikMetadata invalidationKeys
(tableInputs, _, _) = unzip3 $ map mkTableInputs $ OMap.elems $ _smTables sourceMetadata
!namingConv = if isNamingConventionEnabled then getNamingConvention (_smCustomization sourceMetadata) defaultNC else HasuraCase
tablesCoreInfo <-
buildTableCache
-<
( sourceName,
_rsConfig source,
_rsTables source,
tableInputs,
metadataInvalidationKey,
namingConv
)
let tablesMetadata = OMap.elems $ _smTables sourceMetadata
eventTriggers = map (_tmTable &&& OMap.elems . _tmEventTriggers) tablesMetadata
numEventTriggers = sum $ map (length . snd) eventTriggers
sourceConfig = _rsConfig source
(recreateEventTriggers, sourceCatalogMigrationState) <- initCatalogIfNeeded -< (Proxy :: Proxy b, numEventTriggers > 0, sourceConfig)
bindA -< unLogger logger (sourceName, sourceCatalogMigrationState)
let alignTableMap :: HashMap (TableName b) a -> HashMap (TableName b) c -> HashMap (TableName b) (a, c)
alignTableMap = M.intersectionWith (,)
eventTriggerInfoMaps <-
(|
Inc.keyed
( \_ (tableCoreInfo, (_, eventTriggerConfs)) ->
buildTableEventTriggers -< (sourceName, sourceConfig, tableCoreInfo, eventTriggerConfs, metadataInvalidationKey, recreateEventTriggers)
)
|) (tablesCoreInfo `alignTableMap` mapFromL fst eventTriggers)
returnA
-<
Just $
AB.mkAnyBackend @b $
PartiallyResolvedSource sourceMetadata source tablesCoreInfo eventTriggerInfoMaps
)
-<
(exists, (invalidationKeys, defaultNC, isNamingConventionEnabled))
)
|) (M.fromList $ OMap.toList backendInfoAndSourceMetadata)
let partiallyResolvedSources = catMaybes partiallyResolvedSourcesMaybes
-- then we can build the entire source output
-- we need to have the table cache of all sources to build cross-sources relationships
sourcesOutput <-
(|
Inc.keyed
( \_ exists ->
-- Note that it's a bit of a coincidence that
-- 'AB.dispatchAnyBackendArrow' accepts exactly two constraints,
-- and that we happen to want to apply to exactly two
-- constraints.
-- Ideally the function should be able to take an arbitrary
-- number of constraints.
AB.dispatchAnyBackendArrow @BackendMetadata @GetAggregationPredicatesDeps
( proc
( partiallyResolvedSource :: PartiallyResolvedSource b,
(allResolvedSources, remoteSchemaCtxMap, orderedRoles)
)
-> do
let PartiallyResolvedSource sourceMetadata resolvedSource tablesInfo eventTriggers = partiallyResolvedSource
ResolvedSource sourceConfig _sourceCustomization tablesMeta functionsMeta scalars = resolvedSource
so <-
Inc.cache buildSource
-<
( allResolvedSources,
sourceMetadata,
sourceConfig,
tablesInfo,
eventTriggers,
tablesMeta,
functionsMeta,
remoteSchemaCtxMap,
orderedRoles
)
returnA -< (AB.mkAnyBackend so, BackendMap.singleton scalars)
)
-<
( exists,
(partiallyResolvedSources, remoteSchemaCtxMap, orderedRoles)
)
)
|) partiallyResolvedSources
remoteSchemaCache <-
interpretWriter
-< for remoteSchemaMap \(partiallyResolvedRemoteSchemaCtx, metadataObj) -> do
let remoteSchemaIntrospection = irDoc $ _rscIntroOriginal partiallyResolvedRemoteSchemaCtx
resolvedSchemaCtx <- for partiallyResolvedRemoteSchemaCtx \PartiallyResolvedRemoteRelationship {..} ->
buildRemoteSchemaRemoteRelationship partiallyResolvedSources remoteSchemaCtxMap (_rscName partiallyResolvedRemoteSchemaCtx) remoteSchemaIntrospection _prrrTypeName _prrrDefinition
pure $ (catMaybes resolvedSchemaCtx, metadataObj)
allow custom mutations through actions (#3042) * basic doc for actions * custom_types, sync and async actions * switch to graphql-parser-hs on github * update docs * metadata import/export * webhook calls are now supported * relationships in sync actions * initialise.sql is now in sync with the migration file * fix metadata tests * allow specifying arguments of actions * fix blacklist check on check_build_worthiness job * track custom_types and actions related tables * handlers are now triggered on async actions * default to pgjson unless a field is involved in relationships, for generating definition list * use 'true' for action filter for non admin role * fix create_action_permission sql query * drop permissions when dropping an action * add a hdb_role view (and relationships) to fetch all roles in the system * rename 'webhook' key in action definition to 'handler' * allow templating actions wehook URLs with env vars * add 'update_action' /v1/query type * allow forwarding client headers by setting `forward_client_headers` in action definition * add 'headers' configuration in action definition * handle webhook error response based on status codes * support array relationships for custom types * implement single row mutation, see https://github.com/hasura/graphql-engine/issues/3731 * single row mutation: rename 'pk_columns' -> 'columns' and no-op refactor * use top level primary key inputs for delete_by_pk & account select permissions for single row mutations * use only REST semantics to resolve the webhook response * use 'pk_columns' instead of 'columns' for update_by_pk input * add python basic tests for single row mutations * add action context (name) in webhook payload * Async action response is accessible for non admin roles only if the request session vars equals to action's * clean nulls, empty arrays for actions, custom types in export metadata * async action mutation returns only the UUID of the action * unit tests for URL template parser * Basic sync actions python tests * fix output in async query & add async tests * add admin secret header in async actions python test * document async action architecture in Resolve/Action.hs file * support actions returning array of objects * tests for list type response actions * update docs with actions and custom types metadata API reference * update actions python tests as per #f8e1330 Co-authored-by: Tirumarai Selvan <tirumarai.selvan@gmail.com> Co-authored-by: Aravind Shankar <face11301@gmail.com> Co-authored-by: Rakesh Emmadi <12475069+rakeshkky@users.noreply.github.com>
2020-02-13 20:38:23 +03:00
-- actions
(actionCache, annotatedCustomTypes) <-
interpretWriter
-< do
-- custom types
let scalarsMap = mconcat $ map snd $ M.elems sourcesOutput
sourcesCache = M.map fst sourcesOutput
actionList = OMap.elems actions
maybeResolvedCustomTypes <-
withRecordInconsistencyM (MetadataObject MOCustomTypes $ toJSON customTypes) $
resolveCustomTypes sourcesCache customTypes scalarsMap
case maybeResolvedCustomTypes of
Just resolvedCustomTypes -> do
actionCache' <- buildActions resolvedCustomTypes scalarsMap orderedRoles actionList
pure (actionCache', resolvedCustomTypes)
-- If the custom types themselves are inconsistent, we cant really do
-- anything with actions, so just mark them all inconsistent.
Nothing -> do
recordInconsistenciesM
(map mkActionMetadataObject actionList)
"custom types are inconsistent"
pure (mempty, mempty)
allow custom mutations through actions (#3042) * basic doc for actions * custom_types, sync and async actions * switch to graphql-parser-hs on github * update docs * metadata import/export * webhook calls are now supported * relationships in sync actions * initialise.sql is now in sync with the migration file * fix metadata tests * allow specifying arguments of actions * fix blacklist check on check_build_worthiness job * track custom_types and actions related tables * handlers are now triggered on async actions * default to pgjson unless a field is involved in relationships, for generating definition list * use 'true' for action filter for non admin role * fix create_action_permission sql query * drop permissions when dropping an action * add a hdb_role view (and relationships) to fetch all roles in the system * rename 'webhook' key in action definition to 'handler' * allow templating actions wehook URLs with env vars * add 'update_action' /v1/query type * allow forwarding client headers by setting `forward_client_headers` in action definition * add 'headers' configuration in action definition * handle webhook error response based on status codes * support array relationships for custom types * implement single row mutation, see https://github.com/hasura/graphql-engine/issues/3731 * single row mutation: rename 'pk_columns' -> 'columns' and no-op refactor * use top level primary key inputs for delete_by_pk & account select permissions for single row mutations * use only REST semantics to resolve the webhook response * use 'pk_columns' instead of 'columns' for update_by_pk input * add python basic tests for single row mutations * add action context (name) in webhook payload * Async action response is accessible for non admin roles only if the request session vars equals to action's * clean nulls, empty arrays for actions, custom types in export metadata * async action mutation returns only the UUID of the action * unit tests for URL template parser * Basic sync actions python tests * fix output in async query & add async tests * add admin secret header in async actions python test * document async action architecture in Resolve/Action.hs file * support actions returning array of objects * tests for list type response actions * update docs with actions and custom types metadata API reference * update actions python tests as per #f8e1330 Co-authored-by: Tirumarai Selvan <tirumarai.selvan@gmail.com> Co-authored-by: Aravind Shankar <face11301@gmail.com> Co-authored-by: Rakesh Emmadi <12475069+rakeshkky@users.noreply.github.com>
2020-02-13 20:38:23 +03:00
Avoid GraphQL schema rebuild when changing irrelevant Metadata This increases the speed of `create_query_collection` and `add_collection_to_allowlist` by a factor ~~10~~ 65, by caching the in-memory GraphQL schema. This speedup also applies more broadly to Metadata changes relating to: - allowlists - query collections - cron triggers - REST endpoints - API limits - metrics config - GraphQL introspection options - TLS allow lists - OpenTelemetry When is construction of the in-memory GraphQL schema cached between Metadata operations? Before this PR, **never**! It's rebuilt fully, for every role, on every Metadata operation. However, there are many Metadata operations that don't influence the GraphQL schema. So we should be caching its construction. The `Hasura.Incremental` framework allows us to cache such constructions: whenever we have an arrow `Rule m a b`, where `a` is the input to the arrow and `b` the output, we can use the `Inc.cache` combinator to obtain a new arrow which is only re-executed when the input `a` changes in a material way. To test this, `a` needs an `Eq` instance. (Before hasura/graphql-engine-mono#6877, this was a `Cacheable` type class which has now been removed.) We can't simply apply `Inc.cache` to the "Steps 3 and 4" in `buildSchemaCacheRule`, because the inputs (components of `BuildOutputs` such as `SourceCache`) don't have an `Eq` instance. So the changes to `buildSchemaCacheRule` restructure the code so that the input to "Step 1", namely the Metadata, can be used as a caching key instead, so that `Inc.cache` can be applied to the whole sequence of steps. That works to cache construction of the GraphQL schema, but it means that now only those Metadata operations that _don't_ influence any of the products of steps 1-4 can use a cached build of the GraphQL schema. The most important intermediate product is `BuildOutputs`. So now the exercise becomes to minimize the amount of stuff stored in `BuildOutputs`, so that as many Metadata operations as possible can be handled outside of the codepath that produces a GraphQL schema. Per hasura/graphql-engine-mono#6609, the `BuildOutputs` structure is too big, and stores things unnecessarily. Refer to the PR description there for reasoning - the same logic applies to this PR, and simply goes a few steps further. In doing so, it can benefit from hasura/graphql-engine-mono#6765, which allows us to verify at compile time that certain Schema Cache building steps _don't_ generate "Metadata dependencies". If a certain Metadata dependency is never generated, we don't need to handle that case in `deleteMetadataObject`. Thus such intermediate products don't need to be passed through `resolveDependencies`, and thus they don't need to be stored in `BuildOutputs`, and thus their rebuild won't trigger a GraphQL schema rebuild. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6613 GitOrigin-RevId: 27d2e69d3461bd4c32f08febef9995c0369fab3a
2022-11-22 15:11:32 +03:00
returnA
-<
BuildOutputs
{ _boSources = M.map fst sourcesOutput,
_boActions = actionCache,
_boRemoteSchemas = remoteSchemaCache,
_boCustomTypes = annotatedCustomTypes,
_boRoles = mapFromL _rRoleName $ _unOrderedRoles orderedRoles,
_boBackendCache = backendCache
}
buildOpenTelemetry ::
MonadWriter (Seq (Either InconsistentMetadata md)) m =>
OpenTelemetryConfig ->
m OpenTelemetryInfo
buildOpenTelemetry openTelemetryConfig = do
case _ocStatus openTelemetryConfig of
OtelDisabled ->
-- Disable all components if OpenTelemetry export not enabled
pure $ OpenTelemetryInfo Nothing Nothing
OtelEnabled -> do
mOtelExporterInfo <-
let exporterOtlp = _ocExporterOtlp openTelemetryConfig
in withRecordInconsistencyM (MetadataObject (MOOpenTelemetry OtelSubobjectExporterOtlp) (toJSON exporterOtlp)) $
liftEither $
parseOtelExporterConfig env exporterOtlp
mOtelBatchSpanProcessorInfo <-
let batchSpanProcessor = _ocBatchSpanProcessor openTelemetryConfig
in withRecordInconsistencyM (MetadataObject (MOOpenTelemetry OtelSubobjectBatchSpanProcessor) (toJSON batchSpanProcessor)) $
liftEither $
parseOtelBatchSpanProcessorConfig batchSpanProcessor
pure $
OpenTelemetryInfo
mOtelExporterInfo
-- Disable data types if they are not in the enabled set
( if OtelTraces `S.member` _ocEnabledDataTypes openTelemetryConfig
then mOtelBatchSpanProcessorInfo
else Nothing
)
buildRESTEndpoints ::
MonadWriter (Seq (Either InconsistentMetadata md)) m =>
QueryCollections ->
[CreateEndpoint] ->
m (HashMap EndpointName (EndpointMetadata GQLQueryWithText))
buildRESTEndpoints collections endpoints = buildInfoMapM _ceName mkEndpointMetadataObject buildEndpoint endpoints
where
mkEndpointMetadataObject createEndpoint@EndpointMetadata {..} =
let objectId = MOEndpoint _ceName
in MetadataObject objectId (toJSON createEndpoint)
buildEndpoint createEndpoint@EndpointMetadata {..} = do
let -- QueryReference collName queryName = _edQuery endpoint
addContext err = "in endpoint " <> toTxt _ceName <> ": " <> err
withRecordInconsistencyM (mkEndpointMetadataObject createEndpoint) $ modifyErr addContext $ resolveEndpoint collections createEndpoint
resolveEndpoint ::
QErrM m =>
InsOrdHashMap CollectionName CreateCollection ->
EndpointMetadata QueryReference ->
m (EndpointMetadata GQLQueryWithText)
resolveEndpoint collections = traverse $ \(QueryReference collName queryName) -> do
collection <-
onNothing
(OMap.lookup collName collections)
(throw400 NotExists $ "collection with name " <> toTxt collName <> " does not exist")
listedQuery <-
flip
onNothing
( throw400 NotExists $
"query with name "
<> toTxt queryName
<> " does not exist in collection "
<> toTxt collName
)
$ find ((== queryName) . _lqName) (_cdQueries (_ccDefinition collection))
let lq@(GQLQueryWithText lqq) = _lqQuery listedQuery
ds = G.getExecutableDefinitions $ unGQLQuery $ snd lqq
case ds of
[G.ExecutableDefinitionOperation (G.OperationDefinitionTyped d)]
| G._todType d == G.OperationTypeSubscription ->
throw405 $ "query with name " <> toTxt queryName <> " is a subscription"
| otherwise -> pure ()
[] -> throw400 BadRequest $ "query with name " <> toTxt queryName <> " has no definitions."
_ -> throw400 BadRequest $ "query with name " <> toTxt queryName <> " has multiple definitions."
pure lq
mkEventTriggerMetadataObject ::
forall b a c.
Backend b =>
(a, SourceName, c, TableName b, RecreateEventTriggers, EventTriggerConf b) ->
MetadataObject
mkEventTriggerMetadataObject (_, source, _, table, _, eventTriggerConf) =
let objectId =
MOSourceObjId source $
AB.mkAnyBackend $
SMOTableObj @b table $
MTOTrigger $
etcName eventTriggerConf
definition = object ["table" .= table, "configuration" .= eventTriggerConf]
in MetadataObject objectId definition
mkCronTriggerMetadataObject catalogCronTrigger =
let definition = toJSON catalogCronTrigger
in MetadataObject
(MOCronTrigger (ctName catalogCronTrigger))
definition
mkActionMetadataObject (ActionMetadata name comment defn _) =
MetadataObject (MOAction name) (toJSON $ CreateAction name defn comment)
mkInheritedRoleMetadataObject inheritedRole@(Role roleName _) =
MetadataObject (MOInheritedRole roleName) (toJSON inheritedRole)
buildTableEventTriggers ::
forall arr m b.
( ArrowChoice arr,
Inc.ArrowDistribute arr,
ArrowWriter (Seq (Either InconsistentMetadata MetadataDependency)) arr,
Inc.ArrowCache m arr,
MonadIO m,
MonadError QErr m,
MonadBaseControl IO m,
MonadReader BuildReason m,
HasServerConfigCtx m,
BackendMetadata b,
BackendEventTrigger b
) =>
( SourceName,
SourceConfig b,
TableCoreInfoG b (ColumnInfo b) (ColumnInfo b),
[EventTriggerConf b],
Inc.Dependency Inc.InvalidationKey,
RecreateEventTriggers
)
`arr` (EventTriggerInfoMap b)
buildTableEventTriggers = proc (sourceName, sourceConfig, tableInfo, eventTriggerConfs, metadataInvalidationKey, migrationRecreateEventTriggers) ->
buildInfoMap (etcName . (^. _6)) (mkEventTriggerMetadataObject @b) buildEventTrigger
-<
(tableInfo, map (metadataInvalidationKey,sourceName,sourceConfig,_tciName tableInfo,migrationRecreateEventTriggers,) eventTriggerConfs)
where
buildEventTrigger = proc (tableInfo, (metadataInvalidationKey, source, sourceConfig, table, migrationRecreateEventTriggers, eventTriggerConf)) -> do
let triggerName = etcName eventTriggerConf
server: provide an option to enable event triggers on logically replicated tables ## Description ✍️ This PR introduces a new feature to enable/disable event triggers during logical replication of table data for PostgreSQL and MS-SQL data sources. We introduce a new field `trigger_on_replication` in the `*_create_event_trigger` metadata API. By default the event triggers will not fire for logical data replication. ## Changelog ✍️ __Component__ : server __Type__: feature __Product__: community-edition ### Short Changelog Add option to enable/disable event triggers on logically replicated tables ### Related Issues ✍ https://github.com/hasura/graphql-engine/issues/8814 https://hasurahq.atlassian.net/browse/GS-252 ### Solution and Design - By default, triggers do **not** fire when the session mode is `replica` in Postgres, so if the `triggerOnReplication` is set to `true` for an event trigger we run the query `ALTER TABLE #{tableTxt} ENABLE ALWAYS TRIGGER #{triggerNameTxt};` so that the trigger fires always irrespective of the `session_replication_role` - By default, triggers do fire in case of replication in MS-SQL, so if the `triggerOnReplication` is set to `false` for an event trigger we add a clause `NOT FOR REPLICATION` to the the SQL when the trigger is created/altered, which sets the `is_not_for_replication` for the trigger as `true` and it does not fire during logical replication. ### Steps to test and verify ✍ - Run hspec integration tests for HGE ## Server checklist ✍ ### Metadata ✍ Does this PR add a new Metadata feature? - ✅ Yes - Does `export_metadata`/`replace_metadata` supports the new metadata added? - ✅ PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6953 Co-authored-by: Puru Gupta <32328846+purugupta99@users.noreply.github.com> Co-authored-by: Sean Park-Ross <94021366+seanparkross@users.noreply.github.com> GitOrigin-RevId: 92731328a2bbdcad2302c829f26f9acb33c36135
2022-11-29 20:41:41 +03:00
triggerOnReplication = etcTriggerOnReplication eventTriggerConf
metadataObject = mkEventTriggerMetadataObject @b (metadataInvalidationKey, source, sourceConfig, table, migrationRecreateEventTriggers, eventTriggerConf)
schemaObjectId =
SOSourceObj source $
AB.mkAnyBackend $
SOITableObj @b table $
TOTrigger triggerName
addTriggerContext e = "in event trigger " <> triggerName <<> ": " <> e
buildReason <- bindA -< ask
let reloadMetadataRecreateEventTrigger =
case buildReason of
CatalogSync -> RETDoNothing
CatalogUpdate Nothing -> RETDoNothing
CatalogUpdate (Just sources) -> if source `elem` sources then RETRecreate else RETDoNothing
(|
withRecordInconsistency
( do
(info, dependencies) <- bindErrorA -< modifyErr (addTableContext @b table . addTriggerContext) $ buildEventTriggerInfo @b env source table eventTriggerConf
serverConfigCtx <- bindA -< askServerConfigCtx
let isCatalogUpdate =
case buildReason of
CatalogUpdate _ -> True
CatalogSync -> False
tableColumns = M.elems $ _tciFieldInfoMap tableInfo
if ( _sccMaintenanceMode serverConfigCtx == MaintenanceModeDisabled
&& _sccReadOnlyMode serverConfigCtx == ReadOnlyModeDisabled
)
then do
bindA
-<
when (reloadMetadataRecreateEventTrigger == RETRecreate) $
-- This is the case when the user sets `recreate_event_triggers`
-- to `true` in `reload_metadata`, in this case, we recreate
-- the SQL trigger by force, even if it may not be necessary
liftEitherM $
createTableEventTrigger
@b
serverConfigCtx
sourceConfig
table
tableColumns
triggerName
server: provide an option to enable event triggers on logically replicated tables ## Description ✍️ This PR introduces a new feature to enable/disable event triggers during logical replication of table data for PostgreSQL and MS-SQL data sources. We introduce a new field `trigger_on_replication` in the `*_create_event_trigger` metadata API. By default the event triggers will not fire for logical data replication. ## Changelog ✍️ __Component__ : server __Type__: feature __Product__: community-edition ### Short Changelog Add option to enable/disable event triggers on logically replicated tables ### Related Issues ✍ https://github.com/hasura/graphql-engine/issues/8814 https://hasurahq.atlassian.net/browse/GS-252 ### Solution and Design - By default, triggers do **not** fire when the session mode is `replica` in Postgres, so if the `triggerOnReplication` is set to `true` for an event trigger we run the query `ALTER TABLE #{tableTxt} ENABLE ALWAYS TRIGGER #{triggerNameTxt};` so that the trigger fires always irrespective of the `session_replication_role` - By default, triggers do fire in case of replication in MS-SQL, so if the `triggerOnReplication` is set to `false` for an event trigger we add a clause `NOT FOR REPLICATION` to the the SQL when the trigger is created/altered, which sets the `is_not_for_replication` for the trigger as `true` and it does not fire during logical replication. ### Steps to test and verify ✍ - Run hspec integration tests for HGE ## Server checklist ✍ ### Metadata ✍ Does this PR add a new Metadata feature? - ✅ Yes - Does `export_metadata`/`replace_metadata` supports the new metadata added? - ✅ PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6953 Co-authored-by: Puru Gupta <32328846+purugupta99@users.noreply.github.com> Co-authored-by: Sean Park-Ross <94021366+seanparkross@users.noreply.github.com> GitOrigin-RevId: 92731328a2bbdcad2302c829f26f9acb33c36135
2022-11-29 20:41:41 +03:00
triggerOnReplication
(etcDefinition eventTriggerConf)
(_tciPrimaryKey tableInfo)
if isCatalogUpdate || migrationRecreateEventTriggers == RETRecreate
then do
recreateTriggerIfNeeded
-<
( table,
tableColumns,
triggerName,
server: provide an option to enable event triggers on logically replicated tables ## Description ✍️ This PR introduces a new feature to enable/disable event triggers during logical replication of table data for PostgreSQL and MS-SQL data sources. We introduce a new field `trigger_on_replication` in the `*_create_event_trigger` metadata API. By default the event triggers will not fire for logical data replication. ## Changelog ✍️ __Component__ : server __Type__: feature __Product__: community-edition ### Short Changelog Add option to enable/disable event triggers on logically replicated tables ### Related Issues ✍ https://github.com/hasura/graphql-engine/issues/8814 https://hasurahq.atlassian.net/browse/GS-252 ### Solution and Design - By default, triggers do **not** fire when the session mode is `replica` in Postgres, so if the `triggerOnReplication` is set to `true` for an event trigger we run the query `ALTER TABLE #{tableTxt} ENABLE ALWAYS TRIGGER #{triggerNameTxt};` so that the trigger fires always irrespective of the `session_replication_role` - By default, triggers do fire in case of replication in MS-SQL, so if the `triggerOnReplication` is set to `false` for an event trigger we add a clause `NOT FOR REPLICATION` to the the SQL when the trigger is created/altered, which sets the `is_not_for_replication` for the trigger as `true` and it does not fire during logical replication. ### Steps to test and verify ✍ - Run hspec integration tests for HGE ## Server checklist ✍ ### Metadata ✍ Does this PR add a new Metadata feature? - ✅ Yes - Does `export_metadata`/`replace_metadata` supports the new metadata added? - ✅ PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6953 Co-authored-by: Puru Gupta <32328846+purugupta99@users.noreply.github.com> Co-authored-by: Sean Park-Ross <94021366+seanparkross@users.noreply.github.com> GitOrigin-RevId: 92731328a2bbdcad2302c829f26f9acb33c36135
2022-11-29 20:41:41 +03:00
triggerOnReplication,
etcDefinition eventTriggerConf,
sourceConfig,
(_tciPrimaryKey tableInfo)
)
-- We check if the SQL triggers for the event triggers
-- are present. If any SQL triggers are missing, those are
-- created.
bindA
-<
createMissingSQLTriggers
sourceConfig
table
(tableColumns, _tciPrimaryKey tableInfo)
triggerName
server: provide an option to enable event triggers on logically replicated tables ## Description ✍️ This PR introduces a new feature to enable/disable event triggers during logical replication of table data for PostgreSQL and MS-SQL data sources. We introduce a new field `trigger_on_replication` in the `*_create_event_trigger` metadata API. By default the event triggers will not fire for logical data replication. ## Changelog ✍️ __Component__ : server __Type__: feature __Product__: community-edition ### Short Changelog Add option to enable/disable event triggers on logically replicated tables ### Related Issues ✍ https://github.com/hasura/graphql-engine/issues/8814 https://hasurahq.atlassian.net/browse/GS-252 ### Solution and Design - By default, triggers do **not** fire when the session mode is `replica` in Postgres, so if the `triggerOnReplication` is set to `true` for an event trigger we run the query `ALTER TABLE #{tableTxt} ENABLE ALWAYS TRIGGER #{triggerNameTxt};` so that the trigger fires always irrespective of the `session_replication_role` - By default, triggers do fire in case of replication in MS-SQL, so if the `triggerOnReplication` is set to `false` for an event trigger we add a clause `NOT FOR REPLICATION` to the the SQL when the trigger is created/altered, which sets the `is_not_for_replication` for the trigger as `true` and it does not fire during logical replication. ### Steps to test and verify ✍ - Run hspec integration tests for HGE ## Server checklist ✍ ### Metadata ✍ Does this PR add a new Metadata feature? - ✅ Yes - Does `export_metadata`/`replace_metadata` supports the new metadata added? - ✅ PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6953 Co-authored-by: Puru Gupta <32328846+purugupta99@users.noreply.github.com> Co-authored-by: Sean Park-Ross <94021366+seanparkross@users.noreply.github.com> GitOrigin-RevId: 92731328a2bbdcad2302c829f26f9acb33c36135
2022-11-29 20:41:41 +03:00
triggerOnReplication
(etcDefinition eventTriggerConf)
else returnA -< ()
else returnA -< ()
recordDependencies -< (metadataObject, schemaObjectId, dependencies)
returnA -< info
)
|) metadataObject
recreateTriggerIfNeeded =
-- using `Inc.cache` here means that the response will be cached for the given output and the
-- next time this arrow recieves the same input, the cached response will be returned and the
-- computation will not be done again.
Inc.cache
proc
( tableName,
tableColumns,
triggerName,
server: provide an option to enable event triggers on logically replicated tables ## Description ✍️ This PR introduces a new feature to enable/disable event triggers during logical replication of table data for PostgreSQL and MS-SQL data sources. We introduce a new field `trigger_on_replication` in the `*_create_event_trigger` metadata API. By default the event triggers will not fire for logical data replication. ## Changelog ✍️ __Component__ : server __Type__: feature __Product__: community-edition ### Short Changelog Add option to enable/disable event triggers on logically replicated tables ### Related Issues ✍ https://github.com/hasura/graphql-engine/issues/8814 https://hasurahq.atlassian.net/browse/GS-252 ### Solution and Design - By default, triggers do **not** fire when the session mode is `replica` in Postgres, so if the `triggerOnReplication` is set to `true` for an event trigger we run the query `ALTER TABLE #{tableTxt} ENABLE ALWAYS TRIGGER #{triggerNameTxt};` so that the trigger fires always irrespective of the `session_replication_role` - By default, triggers do fire in case of replication in MS-SQL, so if the `triggerOnReplication` is set to `false` for an event trigger we add a clause `NOT FOR REPLICATION` to the the SQL when the trigger is created/altered, which sets the `is_not_for_replication` for the trigger as `true` and it does not fire during logical replication. ### Steps to test and verify ✍ - Run hspec integration tests for HGE ## Server checklist ✍ ### Metadata ✍ Does this PR add a new Metadata feature? - ✅ Yes - Does `export_metadata`/`replace_metadata` supports the new metadata added? - ✅ PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6953 Co-authored-by: Puru Gupta <32328846+purugupta99@users.noreply.github.com> Co-authored-by: Sean Park-Ross <94021366+seanparkross@users.noreply.github.com> GitOrigin-RevId: 92731328a2bbdcad2302c829f26f9acb33c36135
2022-11-29 20:41:41 +03:00
triggerOnReplication,
triggerDefinition,
sourceConfig,
primaryKey
)
-> do
bindA
-< do
serverConfigCtx <- askServerConfigCtx
liftEitherM $
createTableEventTrigger @b
serverConfigCtx
sourceConfig
tableName
tableColumns
triggerName
server: provide an option to enable event triggers on logically replicated tables ## Description ✍️ This PR introduces a new feature to enable/disable event triggers during logical replication of table data for PostgreSQL and MS-SQL data sources. We introduce a new field `trigger_on_replication` in the `*_create_event_trigger` metadata API. By default the event triggers will not fire for logical data replication. ## Changelog ✍️ __Component__ : server __Type__: feature __Product__: community-edition ### Short Changelog Add option to enable/disable event triggers on logically replicated tables ### Related Issues ✍ https://github.com/hasura/graphql-engine/issues/8814 https://hasurahq.atlassian.net/browse/GS-252 ### Solution and Design - By default, triggers do **not** fire when the session mode is `replica` in Postgres, so if the `triggerOnReplication` is set to `true` for an event trigger we run the query `ALTER TABLE #{tableTxt} ENABLE ALWAYS TRIGGER #{triggerNameTxt};` so that the trigger fires always irrespective of the `session_replication_role` - By default, triggers do fire in case of replication in MS-SQL, so if the `triggerOnReplication` is set to `false` for an event trigger we add a clause `NOT FOR REPLICATION` to the the SQL when the trigger is created/altered, which sets the `is_not_for_replication` for the trigger as `true` and it does not fire during logical replication. ### Steps to test and verify ✍ - Run hspec integration tests for HGE ## Server checklist ✍ ### Metadata ✍ Does this PR add a new Metadata feature? - ✅ Yes - Does `export_metadata`/`replace_metadata` supports the new metadata added? - ✅ PR-URL: https://github.com/hasura/graphql-engine-mono/pull/6953 Co-authored-by: Puru Gupta <32328846+purugupta99@users.noreply.github.com> Co-authored-by: Sean Park-Ross <94021366+seanparkross@users.noreply.github.com> GitOrigin-RevId: 92731328a2bbdcad2302c829f26f9acb33c36135
2022-11-29 20:41:41 +03:00
triggerOnReplication
triggerDefinition
primaryKey
buildCronTriggers ::
MonadWriter (Seq (Either InconsistentMetadata md)) m =>
[CronTriggerMetadata] ->
m (HashMap TriggerName CronTriggerInfo)
buildCronTriggers = buildInfoMapM ctName mkCronTriggerMetadataObject buildCronTrigger
where
buildCronTrigger cronTrigger = do
let triggerName = triggerNameToTxt $ ctName cronTrigger
addCronTriggerContext e = "in cron trigger " <> triggerName <> ": " <> e
withRecordInconsistencyM (mkCronTriggerMetadataObject cronTrigger) $
modifyErr addCronTriggerContext $
resolveCronTrigger env cronTrigger
buildInheritedRoles ::
MonadWriter (Seq (Either InconsistentMetadata MetadataDependency)) m =>
HashSet RoleName ->
[InheritedRole] ->
m (HashMap RoleName Role)
buildInheritedRoles allRoles = buildInfoMapM _rRoleName mkInheritedRoleMetadataObject buildInheritedRole
where
buildInheritedRole inheritedRole = do
let addInheritedRoleContext e = "in inherited role " <> roleNameToTxt (_rRoleName inheritedRole) <> ": " <> e
metadataObject = mkInheritedRoleMetadataObject inheritedRole
schemaObject = SORole $ _rRoleName inheritedRole
withRecordInconsistencyM metadataObject $ modifyErr addInheritedRoleContext do
(resolvedInheritedRole, dependencies) <- resolveInheritedRole allRoles inheritedRole
recordDependenciesM metadataObject schemaObject dependencies
pure resolvedInheritedRole
buildActions ::
MonadWriter (Seq (Either InconsistentMetadata MetadataDependency)) m =>
AnnotatedCustomTypes ->
BackendMap ScalarMap ->
OrderedRoles ->
[ActionMetadata] ->
m (HashMap ActionName ActionInfo)
buildActions resolvedCustomTypes scalarsMap orderedRoles = buildInfoMapM _amName mkActionMetadataObject buildAction
where
buildAction action@(ActionMetadata name comment def actionPermissions) = do
let addActionContext e = "in action " <> name <<> "; " <> e
permissionInfos = map (ActionPermissionInfo . _apmRole) actionPermissions
metadataPermissionMap = mapFromL _apiRole permissionInfos
permissionsMap = mkBooleanPermissionMap ActionPermissionInfo metadataPermissionMap orderedRoles
outputType = unGraphQLType $ _adOutputType def
withRecordInconsistencyM (mkActionMetadataObject action) $ modifyErr addActionContext do
(resolvedDef, outObject) <- resolveAction env resolvedCustomTypes def scalarsMap
let forwardClientHeaders = _adForwardClientHeaders resolvedDef
return $ ActionInfo name (outputType, outObject) resolvedDef permissionsMap forwardClientHeaders comment
buildRemoteSchemaRemoteRelationship ::
MonadWriter (Seq (Either InconsistentMetadata MetadataDependency)) m =>
HashMap SourceName (AB.AnyBackend PartiallyResolvedSource) ->
PartiallyResolvedRemoteSchemaMap ->
RemoteSchemaName ->
RemoteSchemaIntrospection ->
G.Name ->
RemoteRelationship ->
m (Maybe (RemoteFieldInfo G.Name))
buildRemoteSchemaRemoteRelationship allSources remoteSchemaMap remoteSchema remoteSchemaIntrospection typeName rr@RemoteRelationship {..} = do
let metadataObject =
MetadataObject (MORemoteSchemaRemoteRelationship remoteSchema typeName _rrName) $
toJSON $
CreateRemoteSchemaRemoteRelationship remoteSchema typeName _rrName _rrDefinition
schemaObj = SORemoteSchemaRemoteRelationship remoteSchema typeName _rrName
addRemoteRelationshipContext e = "in remote relationship" <> _rrName <<> ": " <> e
-- buildRemoteFieldInfo only knows how to construct dependencies on the RHS of the join condition,
-- so the dependencies on the remote relationship on the LHS entity have to be computed here
lhsDependencies =
-- a direct dependency on the remote schema on which this is defined
[SchemaDependency (SORemoteSchema remoteSchema) DRRemoteRelationship]
withRecordInconsistencyM metadataObject $ modifyErr addRemoteRelationshipContext do
allowedLHSJoinFields <- getRemoteSchemaEntityJoinColumns remoteSchema remoteSchemaIntrospection typeName
(remoteField, rhsDependencies) <-
buildRemoteFieldInfo (remoteSchemaToLHSIdentifier remoteSchema) allowedLHSJoinFields rr allSources remoteSchemaMap
recordDependenciesM metadataObject schemaObj (lhsDependencies <> rhsDependencies)
pure remoteField
data BackendInfoAndSourceMetadata b = BackendInfoAndSourceMetadata
{ _bcasmBackendInfo :: BackendInfo b,
_bcasmSourceMetadata :: SourceMetadata b
}
deriving stock (Generic)
deriving instance (Backend b) => Show (BackendInfoAndSourceMetadata b)
deriving instance (Backend b) => Eq (BackendInfoAndSourceMetadata b)
joinBackendInfosToSources ::
BackendCache ->
InsOrdHashMap SourceName BackendSourceMetadata ->
InsOrdHashMap SourceName (AB.AnyBackend BackendInfoAndSourceMetadata)
joinBackendInfosToSources backendInfos sources =
flip OMap.map sources $ \abSourceMetadata ->
AB.dispatchAnyBackend @Backend (unBackendSourceMetadata abSourceMetadata) $ \(sourceMetadata :: SourceMetadata b) ->
let _bcasmBackendInfo = maybe mempty unBackendInfoWrapper (BackendMap.lookup @b backendInfos)
_bcasmSourceMetadata = sourceMetadata
in AB.mkAnyBackend @b BackendInfoAndSourceMetadata {..}
{- Note [Keep invalidation keys for inconsistent objects]
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
After building the schema cache, we prune InvalidationKeys for objects
that no longer exist in the schema to avoid leaking memory for objects
that have been dropped. However, note that we *dont* want to drop
keys for objects that are simply inconsistent!
Why? The object is still in the metadata, so next time we reload it,
well reprocess that object. We want to reuse the cache if its
definition hasnt changed, but if we dropped the invalidation key, it
will incorrectly be reprocessed (since the invalidation key changed
from present to absent). -}