mirror of
https://github.com/hasura/graphql-engine.git
synced 2024-12-14 17:02:49 +03:00
Remove HasServerConfigCtx
from the schema cache build.
## Description This PR is a incremental step towards achieving the goal of #8344. It is a less ambitious version of #8484. This PR removes all references to `HasServerConfigCtx` from the cache build and removes `ServerConfigCtx` from `CacheBuildParams`, making `ServerConfigCtx` an argument being passed around manually instead. This has several benefits: by making it an arrow argument, we now properly integrate the fields that change over time in the dependency framework, as they should be, and we can clean up some of the top-level app code. ## Implementation In practice, this PR introduces a `HasServerConfigCtx` instance for `CacheRWT`, the monad we use to build the cache, so we can retrieve the `ServerConfigCtx` in the implementation of `CacheRWM`. This contributes to reducing the amount of `HasServerConfigCtx` in the code: we can remove `SchemaUpdateT` altogether, and we can remove the `HasServerConfigCtx` instance of `Handler`. This makes `HasServerConfigCtx` almost **an implementation detail of the Metadata API**. This first step is enough to achieve the goal of #8344: we can now build the schema cache in the app monad, since we no longer rely on `HasServerConfigCtx` to build it. ## Drawbacks This PR does not attempt to remove the use of `ServerConfigCtx` itself in the schema cache build: doing so would make this PR much much bigger. Ideally, to avoid having all the static fields given as arrow-ish arguments to the cache, we could depend on `HasAppEnv` in the cache build, and use `AppContext` as an arrow argument. But making the cache build depend on the full `AppEnv` and `AppContext` creates a lot of circular imports; and since removing `ServerConfigCtx` itself isn't required to achieve #8344, this PR keeps it wholesale and defers cleaning it to a future PR. A negative consequence of this is that we need an `Eq` instance on `ServerConfigCtx`, and that instance is inelegant. ## Future work There are several further steps we can take in parallel after this is merged. First, again, we can make a new version of #8344, removing `CacheBuild`, FINALLY. As for `ServerConfigCtx`, we can split it / rename it to make ad-hoc structures. If it turns out that `ServerConfigCtx` is only ever used for the schema cache build, we could split it between `CacheBuildEnv` and `CacheBuildContext`, which will be subsets of `AppEnv` and `AppContext`, avoiding import loops. PR-URL: https://github.com/hasura/graphql-engine-mono/pull/8509 GitOrigin-RevId: 01b37cc3fd3490d6b117701e22fc4ac88b62b6b5
This commit is contained in:
parent
2d9c8299c2
commit
7aa341944b
@ -602,12 +602,12 @@ buildFirstSchemaCache
|
||||
mssqlSourceResolver
|
||||
metadata
|
||||
httpManager = do
|
||||
let cacheBuildParams = CacheBuildParams httpManager pgSourceResolver mssqlSourceResolver serverConfigCtx
|
||||
let cacheBuildParams = CacheBuildParams httpManager pgSourceResolver mssqlSourceResolver
|
||||
buildReason = CatalogSync
|
||||
result <-
|
||||
runExceptT $
|
||||
runCacheBuild cacheBuildParams $
|
||||
buildRebuildableSchemaCacheWithReason buildReason logger env metadata
|
||||
buildRebuildableSchemaCacheWithReason buildReason logger env metadata serverConfigCtx
|
||||
result `onLeft` \err -> do
|
||||
-- TODO: we used to bundle the first schema cache build with the catalog
|
||||
-- migration, using the same error handler for both, meaning that an
|
||||
|
@ -13,6 +13,9 @@ module Hasura.App.State
|
||||
-- * init functions
|
||||
buildRebuildableAppContext,
|
||||
initSQLGenCtx,
|
||||
|
||||
-- * server config
|
||||
buildServerConfigCtx,
|
||||
)
|
||||
where
|
||||
|
||||
@ -285,3 +288,27 @@ initSQLGenCtx experimentalFeatures stringifyNum dangerousBooleanCollapse =
|
||||
| EFBigQueryStringNumericInput `elem` experimentalFeatures = Options.EnableBigQueryStringNumericInput
|
||||
| otherwise = Options.DisableBigQueryStringNumericInput
|
||||
in SQLGenCtx stringifyNum dangerousBooleanCollapse optimizePermissionFilters bigqueryStringNumericInput
|
||||
|
||||
--------------------------------------------------------------------------------
|
||||
-- server config
|
||||
|
||||
-- | We are trying to slowly get rid of 'HasServerConfigCtx' (and consequently
|
||||
-- of 'ServercConfigtx') in favour of smaller / more specific ad-hoc
|
||||
-- types. However, in the meantime, it is often required to builda
|
||||
-- 'ServerConfigCtx' at the boundary between parts of the code that use it and
|
||||
-- part of the code that use the new 'AppEnv' and 'AppContext'.
|
||||
buildServerConfigCtx :: AppEnv -> AppContext -> ServerConfigCtx
|
||||
buildServerConfigCtx AppEnv {..} AppContext {..} =
|
||||
ServerConfigCtx
|
||||
{ _sccFunctionPermsCtx = acFunctionPermsCtx,
|
||||
_sccRemoteSchemaPermsCtx = acRemoteSchemaPermsCtx,
|
||||
_sccSQLGenCtx = acSQLGenCtx,
|
||||
_sccMaintenanceMode = appEnvEnableMaintenanceMode,
|
||||
_sccExperimentalFeatures = acExperimentalFeatures,
|
||||
_sccEventingMode = appEnvEventingMode,
|
||||
_sccReadOnlyMode = appEnvEnableReadOnlyMode,
|
||||
_sccDefaultNamingConvention = acDefaultNamingConvention,
|
||||
_sccMetadataDefaults = acMetadataDefaults,
|
||||
_sccCheckFeatureFlag = appEnvCheckFeatureFlag,
|
||||
_sccApolloFederationStatus = acApolloFederationStatus
|
||||
}
|
||||
|
@ -233,6 +233,7 @@ createMissingSQLTriggers ::
|
||||
MonadError QErr m,
|
||||
MonadBaseControl IO m
|
||||
) =>
|
||||
ServerConfigCtx ->
|
||||
MSSQLSourceConfig ->
|
||||
TableName ->
|
||||
([ColumnInfo 'MSSQL], Maybe (PrimaryKey 'MSSQL (ColumnInfo 'MSSQL))) ->
|
||||
@ -241,6 +242,7 @@ createMissingSQLTriggers ::
|
||||
TriggerOpsDef 'MSSQL ->
|
||||
m ()
|
||||
createMissingSQLTriggers
|
||||
_serverConfigCtx
|
||||
sourceConfig
|
||||
table@(TableName tableNameText (SchemaName schemaText))
|
||||
(allCols, primaryKeyMaybe)
|
||||
|
@ -203,9 +203,9 @@ createMissingSQLTriggers ::
|
||||
( MonadIO m,
|
||||
MonadError QErr m,
|
||||
MonadBaseControl IO m,
|
||||
HasServerConfigCtx m,
|
||||
Backend ('Postgres pgKind)
|
||||
) =>
|
||||
ServerConfigCtx ->
|
||||
PGSourceConfig ->
|
||||
TableName ('Postgres pgKind) ->
|
||||
([(ColumnInfo ('Postgres pgKind))], Maybe (PrimaryKey ('Postgres pgKind) (ColumnInfo ('Postgres pgKind)))) ->
|
||||
@ -213,15 +213,14 @@ createMissingSQLTriggers ::
|
||||
TriggerOnReplication ->
|
||||
TriggerOpsDef ('Postgres pgKind) ->
|
||||
m ()
|
||||
createMissingSQLTriggers sourceConfig table (allCols, _) triggerName triggerOnReplication opsDefinition = do
|
||||
serverConfigCtx <- askServerConfigCtx
|
||||
createMissingSQLTriggers serverConfigCtx sourceConfig table (allCols, _) triggerName triggerOnReplication opsDefinition = do
|
||||
liftEitherM $
|
||||
runPgSourceWriteTx sourceConfig InternalRawQuery $ do
|
||||
for_ (tdInsert opsDefinition) (doesSQLTriggerExist serverConfigCtx INSERT)
|
||||
for_ (tdUpdate opsDefinition) (doesSQLTriggerExist serverConfigCtx UPDATE)
|
||||
for_ (tdDelete opsDefinition) (doesSQLTriggerExist serverConfigCtx DELETE)
|
||||
for_ (tdInsert opsDefinition) (doesSQLTriggerExist INSERT)
|
||||
for_ (tdUpdate opsDefinition) (doesSQLTriggerExist UPDATE)
|
||||
for_ (tdDelete opsDefinition) (doesSQLTriggerExist DELETE)
|
||||
where
|
||||
doesSQLTriggerExist serverConfigCtx op opSpec = do
|
||||
doesSQLTriggerExist op opSpec = do
|
||||
let opTriggerName = pgTriggerName op triggerName
|
||||
doesOpTriggerFunctionExist <-
|
||||
runIdentity . PG.getRow
|
||||
|
@ -18,8 +18,7 @@ runSetGraphqlSchemaIntrospectionOptions ::
|
||||
SetGraphqlIntrospectionOptions ->
|
||||
m EncJSON
|
||||
runSetGraphqlSchemaIntrospectionOptions introspectionOptions = do
|
||||
let metadataModifier = MetadataModifier $ metaSetGraphqlIntrospectionOptions .~ introspectionOptions
|
||||
withNewInconsistentObjsCheck $
|
||||
buildSchemaCache $
|
||||
MetadataModifier $
|
||||
metaSetGraphqlIntrospectionOptions .~ introspectionOptions
|
||||
buildSchemaCache metadataModifier
|
||||
return successMsg
|
||||
|
@ -141,6 +141,7 @@ buildRebuildableSchemaCache ::
|
||||
Logger Hasura ->
|
||||
Env.Environment ->
|
||||
Metadata ->
|
||||
ServerConfigCtx ->
|
||||
CacheBuild RebuildableSchemaCache
|
||||
buildRebuildableSchemaCache =
|
||||
buildRebuildableSchemaCacheWithReason CatalogSync
|
||||
@ -150,36 +151,48 @@ buildRebuildableSchemaCacheWithReason ::
|
||||
Logger Hasura ->
|
||||
Env.Environment ->
|
||||
Metadata ->
|
||||
ServerConfigCtx ->
|
||||
CacheBuild RebuildableSchemaCache
|
||||
buildRebuildableSchemaCacheWithReason reason logger env metadata = do
|
||||
buildRebuildableSchemaCacheWithReason reason logger env metadata serverConfigCtx = do
|
||||
result <-
|
||||
flip runReaderT reason $
|
||||
Inc.build (buildSchemaCacheRule logger env) (metadata, initialInvalidationKeys, Nothing)
|
||||
Inc.build (buildSchemaCacheRule logger env) (metadata, serverConfigCtx, initialInvalidationKeys, Nothing)
|
||||
|
||||
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 don’t have that yet.
|
||||
CacheRWT (StateT (RebuildableSchemaCache, CacheInvalidations) 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 don’t have
|
||||
-- that yet.
|
||||
--
|
||||
-- The use of 'ReaderT ServerConfigCtx' is only here to avoid manually
|
||||
-- passing the 'ServerConfigCtx' to every function that builds the cache. It
|
||||
-- should ultimately be reduced to 'AppContext', or even better a relevant
|
||||
-- subset thereof.
|
||||
CacheRWT (ReaderT ServerConfigCtx (StateT (RebuildableSchemaCache, CacheInvalidations) m) a)
|
||||
deriving
|
||||
( Functor,
|
||||
Applicative,
|
||||
Monad,
|
||||
MonadIO,
|
||||
MonadReader r,
|
||||
MonadError e,
|
||||
UserInfoM,
|
||||
MonadMetadataStorage,
|
||||
MonadMetadataStorageQueryAPI,
|
||||
Tracing.MonadTrace,
|
||||
HasServerConfigCtx,
|
||||
MonadBase b,
|
||||
MonadBaseControl b,
|
||||
ProvidesNetwork
|
||||
)
|
||||
|
||||
instance Monad m => HasServerConfigCtx (CacheRWT m) where
|
||||
askServerConfigCtx = CacheRWT ask
|
||||
|
||||
instance MonadReader r m => MonadReader r (CacheRWT m) where
|
||||
ask = lift ask
|
||||
local f (CacheRWT m) = CacheRWT $ mapReaderT (local f) m
|
||||
|
||||
instance (MonadEventLogCleanup m) => MonadEventLogCleanup (CacheRWT m) where
|
||||
runLogCleaner conf = lift $ runLogCleaner conf
|
||||
generateCleanupSchedules sourceInfo triggerName cleanupConfig = lift $ generateCleanupSchedules sourceInfo triggerName cleanupConfig
|
||||
@ -189,15 +202,18 @@ instance (MonadGetApiTimeLimit m) => MonadGetApiTimeLimit (CacheRWT m) where
|
||||
runGetApiTimeLimit = lift $ runGetApiTimeLimit
|
||||
|
||||
runCacheRWT ::
|
||||
Functor m =>
|
||||
Monad m =>
|
||||
ServerConfigCtx ->
|
||||
RebuildableSchemaCache ->
|
||||
CacheRWT m a ->
|
||||
m (a, RebuildableSchemaCache, CacheInvalidations)
|
||||
runCacheRWT cache (CacheRWT m) =
|
||||
runStateT m (cache, mempty) <&> \(v, (newCache, invalidations)) -> (v, newCache, invalidations)
|
||||
runCacheRWT config cache (CacheRWT m) = do
|
||||
(v, (newCache, invalidations)) <-
|
||||
runStateT (runReaderT m config) (cache, mempty)
|
||||
pure (v, newCache, invalidations)
|
||||
|
||||
instance MonadTrans CacheRWT where
|
||||
lift = CacheRWT . lift
|
||||
lift = CacheRWT . lift . lift
|
||||
|
||||
instance (Monad m) => CacheRM (CacheRWT m) where
|
||||
askSchemaCache = CacheRWT $ gets (lastBuiltSchemaCache . (^. _1))
|
||||
@ -206,20 +222,19 @@ instance
|
||||
( MonadIO m,
|
||||
MonadError QErr m,
|
||||
ProvidesNetwork m,
|
||||
MonadResolveSource m,
|
||||
HasServerConfigCtx m
|
||||
MonadResolveSource m
|
||||
) =>
|
||||
CacheRWM (CacheRWT m)
|
||||
where
|
||||
buildSchemaCacheWithOptions buildReason invalidations metadata = CacheRWT do
|
||||
serverConfigCtx <- ask
|
||||
(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, Nothing)
|
||||
runCacheBuildM $
|
||||
flip runReaderT buildReason $
|
||||
Inc.build rule (metadata, serverConfigCtx, newInvalidationKeys, Nothing)
|
||||
let schemaCache = (Inc.result result) {scMetadataResourceVersion = metadataVersion}
|
||||
prunedInvalidationKeys = pruneInvalidationKeys schemaCache newInvalidationKeys
|
||||
!newCache = RebuildableSchemaCache schemaCache prunedInvalidationKeys (Inc.rebuildRule result)
|
||||
@ -314,20 +329,19 @@ buildSchemaCacheRule ::
|
||||
MonadError QErr m,
|
||||
MonadReader BuildReason m,
|
||||
ProvidesNetwork m,
|
||||
MonadResolveSource m,
|
||||
HasServerConfigCtx m
|
||||
MonadResolveSource m
|
||||
) =>
|
||||
Logger Hasura ->
|
||||
Env.Environment ->
|
||||
(Metadata, InvalidationKeys, Maybe StoredIntrospection) `arr` SchemaCache
|
||||
buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, storedIntrospection) -> do
|
||||
(Metadata, ServerConfigCtx, InvalidationKeys, Maybe StoredIntrospection) `arr` SchemaCache
|
||||
buildSchemaCacheRule logger env = proc (metadataNoDefaults, serverConfigCtx, invalidationKeys, storedIntrospection) -> do
|
||||
invalidationKeysDep <- Inc.newDependency -< invalidationKeys
|
||||
metadataDefaults <- bindA -< askMetadataDefaults
|
||||
let metadata@Metadata {..} = overrideMetadataDefaults metadataNoDefaults metadataDefaults
|
||||
let metadataDefaults = _sccMetadataDefaults serverConfigCtx
|
||||
metadata@Metadata {..} = overrideMetadataDefaults metadataNoDefaults metadataDefaults
|
||||
metadataDep <- Inc.newDependency -< metadata
|
||||
|
||||
(inconsistentObjects, (resolvedOutputs, dependencyInconsistentObjects, resolvedDependencies), ((adminIntrospection, gqlContext, gqlContextUnauth, inconsistentRemoteSchemas), (relayContext, relayContextUnauth))) <-
|
||||
Inc.cache buildOutputsAndSchema -< (metadataDep, invalidationKeysDep, storedIntrospection)
|
||||
Inc.cache buildOutputsAndSchema -< (metadataDep, serverConfigCtx, invalidationKeysDep, storedIntrospection)
|
||||
|
||||
let (resolvedEndpoints, endpointCollectedInfo) = runIdentity $ runWriterT $ buildRESTEndpoints _metaQueryCollections (OMap.elems _metaRestEndpoints)
|
||||
(cronTriggersMap, cronTriggersCollectedInfo) = runIdentity $ runWriterT $ buildCronTriggers (OMap.elems _metaCronTriggers)
|
||||
@ -422,16 +436,15 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
}
|
||||
where
|
||||
-- See Note [Avoiding GraphQL schema rebuilds when changing irrelevant Metadata]
|
||||
buildOutputsAndSchema = proc (metadataDep, invalidationKeysDep, storedIntrospection) -> do
|
||||
(outputs, collectedInfo) <- runWriterA buildAndCollectInfo -< (metadataDep, invalidationKeysDep, storedIntrospection)
|
||||
buildOutputsAndSchema = proc (metadataDep, serverConfigCtx, invalidationKeysDep, storedIntrospection) -> do
|
||||
(outputs, collectedInfo) <- runWriterA buildAndCollectInfo -< (serverConfigCtx, metadataDep, invalidationKeysDep, storedIntrospection)
|
||||
let (inconsistentObjects, unresolvedDependencies) = partitionEithers $ toList collectedInfo
|
||||
out2@(resolvedOutputs, _dependencyInconsistentObjects, _resolvedDependencies) <- resolveDependencies -< (outputs, unresolvedDependencies)
|
||||
out3 <-
|
||||
bindA
|
||||
-< do
|
||||
cxt <- askServerConfigCtx
|
||||
buildGQLContext
|
||||
cxt
|
||||
serverConfigCtx
|
||||
(_boSources resolvedOutputs)
|
||||
(_boRemoteSchemas resolvedOutputs)
|
||||
(_boActions resolvedOutputs)
|
||||
@ -566,19 +579,18 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
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
|
||||
(Proxy b, ServerConfigCtx, Bool, SourceConfig b) `arr` (RecreateEventTriggers, SourceCatalogMigrationState)
|
||||
initCatalogIfNeeded = Inc.cache proc (Proxy, serverConfigCtx, atleastOneTrigger, sourceConfig) -> do
|
||||
bindA
|
||||
-< do
|
||||
if atleastOneTrigger
|
||||
then do
|
||||
maintenanceMode <- _sccMaintenanceMode <$> askServerConfigCtx
|
||||
eventingMode <- _sccEventingMode <$> askServerConfigCtx
|
||||
readOnlyMode <- _sccReadOnlyMode <$> askServerConfigCtx
|
||||
let maintenanceMode = _sccMaintenanceMode serverConfigCtx
|
||||
eventingMode = _sccEventingMode serverConfigCtx
|
||||
readOnlyMode = _sccReadOnlyMode serverConfigCtx
|
||||
|
||||
if
|
||||
-- when safe mode is enabled, don't perform any migrations
|
||||
@ -611,12 +623,12 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
( 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),
|
||||
( ServerConfigCtx,
|
||||
HashMap SourceName (AB.AnyBackend PartiallyResolvedSource),
|
||||
SourceMetadata b,
|
||||
SourceConfig b,
|
||||
HashMap (TableName b) (TableCoreInfoG b (ColumnInfo b) (ColumnInfo b)),
|
||||
@ -627,7 +639,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
OrderedRoles
|
||||
)
|
||||
`arr` (SourceInfo b)
|
||||
buildSource = proc (allSources, sourceMetadata, sourceConfig, tablesRawInfo, eventTriggerInfoMaps, _dbTables, dbFunctions, remoteSchemaMap, orderedRoles) -> do
|
||||
buildSource = proc (serverConfigCtx, allSources, sourceMetadata, sourceConfig, tablesRawInfo, eventTriggerInfoMaps, _dbTables, dbFunctions, remoteSchemaMap, orderedRoles) -> do
|
||||
let SourceMetadata sourceName _backendKind tables functions logicalModels _ queryTagsConfig sourceCustomization _healthCheckConfig = sourceMetadata
|
||||
tablesMetadata = OMap.elems tables
|
||||
(_, nonColumnInputs, permissions) = unzip3 $ map mkTableInputs tablesMetadata
|
||||
@ -664,8 +676,8 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
|
||||
-- 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
|
||||
let !defaultNC = _sccDefaultNamingConvention serverConfigCtx
|
||||
!isNamingConventionEnabled = EFNamingConventions `elem` (_sccExperimentalFeatures serverConfigCtx)
|
||||
!namingConv <-
|
||||
bindA
|
||||
-<
|
||||
@ -748,11 +760,10 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
MonadReader BuildReason m,
|
||||
MonadBaseControl IO m,
|
||||
ProvidesNetwork m,
|
||||
HasServerConfigCtx m,
|
||||
MonadResolveSource m
|
||||
) =>
|
||||
(Inc.Dependency Metadata, Inc.Dependency InvalidationKeys, Maybe StoredIntrospection) `arr` BuildOutputs
|
||||
buildAndCollectInfo = proc (metadataDep, invalidationKeys, storedIntrospection) -> do
|
||||
(ServerConfigCtx, Inc.Dependency Metadata, Inc.Dependency InvalidationKeys, Maybe StoredIntrospection) `arr` BuildOutputs
|
||||
buildAndCollectInfo = proc (serverConfigCtx, metadataDep, invalidationKeys, storedIntrospection) -> do
|
||||
sources <- Inc.dependOn -< Inc.selectD #_metaSources metadataDep
|
||||
remoteSchemas <- Inc.dependOn -< Inc.selectD #_metaRemoteSchemas metadataDep
|
||||
customTypes <- Inc.dependOn -< Inc.selectD #_metaCustomTypes metadataDep
|
||||
@ -788,9 +799,8 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
let remoteSchemaInvalidationKeys = Inc.selectD #_ikRemoteSchemas invalidationKeys
|
||||
remoteSchemaMap <- buildRemoteSchemas env -< ((remoteSchemaInvalidationKeys, orderedRoles, fmap encJToLBS . siRemotes <$> storedIntrospection), OMap.elems remoteSchemas)
|
||||
let remoteSchemaCtxMap = M.map fst remoteSchemaMap
|
||||
|
||||
!defaultNC <- bindA -< _sccDefaultNamingConvention <$> askServerConfigCtx
|
||||
!isNamingConventionEnabled <- bindA -< ((EFNamingConventions `elem`) . _sccExperimentalFeatures) <$> askServerConfigCtx
|
||||
!defaultNC = _sccDefaultNamingConvention serverConfigCtx
|
||||
!isNamingConventionEnabled = EFNamingConventions `elem` (_sccExperimentalFeatures serverConfigCtx)
|
||||
|
||||
let backendInvalidationKeys = Inc.selectD #_ikBackends invalidationKeys
|
||||
backendCache <- resolveBackendCache -< (backendInvalidationKeys, BackendMap.elems backendConfigs)
|
||||
@ -804,7 +814,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
Inc.keyed
|
||||
( \_ exists ->
|
||||
AB.dispatchAnyBackendArrow @BackendMetadata @BackendEventTrigger
|
||||
( proc (backendInfoAndSourceMetadata :: BackendInfoAndSourceMetadata b, (invalidationKeys, storedIntrospection, defaultNC, isNamingConventionEnabled)) -> do
|
||||
( proc (backendInfoAndSourceMetadata :: BackendInfoAndSourceMetadata b, (serverConfigCtx, invalidationKeys, storedIntrospection, defaultNC, isNamingConventionEnabled)) -> do
|
||||
let sourceMetadata = _bcasmSourceMetadata backendInfoAndSourceMetadata
|
||||
sourceName = _smName sourceMetadata
|
||||
sourceInvalidationsKeys = Inc.selectD #_ikSources invalidationKeys
|
||||
@ -833,7 +843,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
eventTriggers = map (_tmTable &&& OMap.elems . _tmEventTriggers) tablesMetadata
|
||||
numEventTriggers = sum $ map (length . snd) eventTriggers
|
||||
|
||||
(recreateEventTriggers, sourceCatalogMigrationState) <- initCatalogIfNeeded -< (Proxy :: Proxy b, numEventTriggers > 0, sourceConfig)
|
||||
(recreateEventTriggers, sourceCatalogMigrationState) <- initCatalogIfNeeded -< (Proxy :: Proxy b, serverConfigCtx, numEventTriggers > 0, sourceConfig)
|
||||
|
||||
bindA -< unLogger logger (sourceName, sourceCatalogMigrationState)
|
||||
|
||||
@ -844,7 +854,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
(|
|
||||
Inc.keyed
|
||||
( \_ (tableCoreInfo, (_, eventTriggerConfs)) ->
|
||||
buildTableEventTriggers -< (sourceName, sourceConfig, tableCoreInfo, eventTriggerConfs, metadataInvalidationKey, recreateEventTriggers)
|
||||
buildTableEventTriggers -< (serverConfigCtx, sourceName, sourceConfig, tableCoreInfo, eventTriggerConfs, metadataInvalidationKey, recreateEventTriggers)
|
||||
)
|
||||
|) (tablesCoreInfo `alignTableMap` mapFromL fst eventTriggers)
|
||||
|
||||
@ -855,7 +865,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
PartiallyResolvedSource sourceMetadata sourceConfig source tablesCoreInfo eventTriggerInfoMaps
|
||||
)
|
||||
-<
|
||||
(exists, (invalidationKeys, storedIntrospection, defaultNC, isNamingConventionEnabled))
|
||||
(exists, (serverConfigCtx, invalidationKeys, storedIntrospection, defaultNC, isNamingConventionEnabled))
|
||||
)
|
||||
|) (M.fromList $ OMap.toList backendInfoAndSourceMetadata)
|
||||
let partiallyResolvedSources = catMaybes partiallyResolvedSourcesMaybes
|
||||
@ -875,7 +885,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
AB.dispatchAnyBackendArrow @BackendMetadata @GetAggregationPredicatesDeps
|
||||
( proc
|
||||
( partiallyResolvedSource :: PartiallyResolvedSource b,
|
||||
(allResolvedSources, remoteSchemaCtxMap, orderedRoles)
|
||||
(serverConfigCtx, allResolvedSources, remoteSchemaCtxMap, orderedRoles)
|
||||
)
|
||||
-> do
|
||||
let PartiallyResolvedSource sourceMetadata sourceConfig introspection tablesInfo eventTriggers = partiallyResolvedSource
|
||||
@ -883,7 +893,8 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
so <-
|
||||
Inc.cache buildSource
|
||||
-<
|
||||
( allResolvedSources,
|
||||
( serverConfigCtx,
|
||||
allResolvedSources,
|
||||
sourceMetadata,
|
||||
sourceConfig,
|
||||
tablesInfo,
|
||||
@ -897,7 +908,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
)
|
||||
-<
|
||||
( exists,
|
||||
(partiallyResolvedSources, remoteSchemaCtxMap, orderedRoles)
|
||||
(serverConfigCtx, partiallyResolvedSources, remoteSchemaCtxMap, orderedRoles)
|
||||
)
|
||||
)
|
||||
|) partiallyResolvedSources
|
||||
@ -1026,9 +1037,9 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
mkEventTriggerMetadataObject ::
|
||||
forall b a c.
|
||||
Backend b =>
|
||||
(a, SourceName, c, TableName b, RecreateEventTriggers, EventTriggerConf b) ->
|
||||
(ServerConfigCtx, a, SourceName, c, TableName b, RecreateEventTriggers, EventTriggerConf b) ->
|
||||
MetadataObject
|
||||
mkEventTriggerMetadataObject (_, source, _, table, _, eventTriggerConf) =
|
||||
mkEventTriggerMetadataObject (_, _, source, _, table, _, eventTriggerConf) =
|
||||
let objectId =
|
||||
MOSourceObjId source $
|
||||
AB.mkAnyBackend $
|
||||
@ -1060,11 +1071,11 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
MonadError QErr m,
|
||||
MonadBaseControl IO m,
|
||||
MonadReader BuildReason m,
|
||||
HasServerConfigCtx m,
|
||||
BackendMetadata b,
|
||||
BackendEventTrigger b
|
||||
) =>
|
||||
( SourceName,
|
||||
( ServerConfigCtx,
|
||||
SourceName,
|
||||
SourceConfig b,
|
||||
TableCoreInfoG b (ColumnInfo b) (ColumnInfo b),
|
||||
[EventTriggerConf b],
|
||||
@ -1072,15 +1083,15 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
RecreateEventTriggers
|
||||
)
|
||||
`arr` (EventTriggerInfoMap b)
|
||||
buildTableEventTriggers = proc (sourceName, sourceConfig, tableInfo, eventTriggerConfs, metadataInvalidationKey, migrationRecreateEventTriggers) ->
|
||||
buildInfoMap (etcName . (^. _6)) (mkEventTriggerMetadataObject @b) buildEventTrigger
|
||||
buildTableEventTriggers = proc (serverConfigCtx, sourceName, sourceConfig, tableInfo, eventTriggerConfs, metadataInvalidationKey, migrationRecreateEventTriggers) ->
|
||||
buildInfoMap (etcName . (^. _7)) (mkEventTriggerMetadataObject @b) buildEventTrigger
|
||||
-<
|
||||
(tableInfo, map (metadataInvalidationKey,sourceName,sourceConfig,_tciName tableInfo,migrationRecreateEventTriggers,) eventTriggerConfs)
|
||||
(tableInfo, map (serverConfigCtx,metadataInvalidationKey,sourceName,sourceConfig,_tciName tableInfo,migrationRecreateEventTriggers,) eventTriggerConfs)
|
||||
where
|
||||
buildEventTrigger = proc (tableInfo, (metadataInvalidationKey, source, sourceConfig, table, migrationRecreateEventTriggers, eventTriggerConf)) -> do
|
||||
buildEventTrigger = proc (tableInfo, (serverConfigCtx, metadataInvalidationKey, source, sourceConfig, table, migrationRecreateEventTriggers, eventTriggerConf)) -> do
|
||||
let triggerName = etcName eventTriggerConf
|
||||
triggerOnReplication = etcTriggerOnReplication eventTriggerConf
|
||||
metadataObject = mkEventTriggerMetadataObject @b (metadataInvalidationKey, source, sourceConfig, table, migrationRecreateEventTriggers, eventTriggerConf)
|
||||
metadataObject = mkEventTriggerMetadataObject @b (serverConfigCtx, metadataInvalidationKey, source, sourceConfig, table, migrationRecreateEventTriggers, eventTriggerConf)
|
||||
schemaObjectId =
|
||||
SOSourceObj source $
|
||||
AB.mkAnyBackend $
|
||||
@ -1097,7 +1108,6 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
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
|
||||
@ -1128,7 +1138,8 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
then do
|
||||
recreateTriggerIfNeeded
|
||||
-<
|
||||
( table,
|
||||
( serverConfigCtx,
|
||||
table,
|
||||
tableColumns,
|
||||
triggerName,
|
||||
triggerOnReplication,
|
||||
@ -1142,6 +1153,7 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
bindA
|
||||
-<
|
||||
createMissingSQLTriggers
|
||||
serverConfigCtx
|
||||
sourceConfig
|
||||
table
|
||||
(tableColumns, _tciPrimaryKey tableInfo)
|
||||
@ -1161,7 +1173,8 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
-- computation will not be done again.
|
||||
Inc.cache
|
||||
proc
|
||||
( tableName,
|
||||
( serverConfigCtx,
|
||||
tableName,
|
||||
tableColumns,
|
||||
triggerName,
|
||||
triggerOnReplication,
|
||||
@ -1172,7 +1185,6 @@ buildSchemaCacheRule logger env = proc (metadataNoDefaults, invalidationKeys, st
|
||||
-> do
|
||||
bindA
|
||||
-< do
|
||||
serverConfigCtx <- askServerConfigCtx
|
||||
liftEitherM $
|
||||
createTableEventTrigger @b
|
||||
serverConfigCtx
|
||||
|
@ -257,8 +257,7 @@ $(makeLenses ''BuildOutputs)
|
||||
data CacheBuildParams = CacheBuildParams
|
||||
{ _cbpManager :: HTTP.Manager,
|
||||
_cbpPGSourceResolver :: SourceResolver ('Postgres 'Vanilla),
|
||||
_cbpMSSQLSourceResolver :: SourceResolver 'MSSQL,
|
||||
_cbpServerConfigCtx :: ServerConfigCtx
|
||||
_cbpMSSQLSourceResolver :: SourceResolver 'MSSQL
|
||||
}
|
||||
|
||||
-- | The monad in which @'RebuildableSchemaCache' is being run
|
||||
@ -277,9 +276,6 @@ newtype CacheBuild a = CacheBuild (ReaderT CacheBuildParams (ExceptT QErr IO) a)
|
||||
instance ProvidesNetwork CacheBuild where
|
||||
askHTTPManager = asks _cbpManager
|
||||
|
||||
instance HasServerConfigCtx CacheBuild where
|
||||
askServerConfigCtx = asks _cbpServerConfigCtx
|
||||
|
||||
instance MonadResolveSource CacheBuild where
|
||||
getPGSourceResolver = asks _cbpPGSourceResolver
|
||||
getMSSQLSourceResolver = asks _cbpMSSQLSourceResolver
|
||||
@ -297,7 +293,6 @@ runCacheBuild params (CacheBuild m) = do
|
||||
runCacheBuildM ::
|
||||
( MonadIO m,
|
||||
MonadError QErr m,
|
||||
HasServerConfigCtx m,
|
||||
MonadResolveSource m,
|
||||
ProvidesNetwork m
|
||||
) =>
|
||||
@ -309,13 +304,12 @@ runCacheBuildM m = do
|
||||
<$> askHTTPManager
|
||||
<*> getPGSourceResolver
|
||||
<*> getMSSQLSourceResolver
|
||||
<*> askServerConfigCtx
|
||||
runCacheBuild params m
|
||||
|
||||
data RebuildableSchemaCache = RebuildableSchemaCache
|
||||
{ lastBuiltSchemaCache :: SchemaCache,
|
||||
_rscInvalidationMap :: InvalidationKeys,
|
||||
_rscRebuild :: Inc.Rule (ReaderT BuildReason CacheBuild) (Metadata, InvalidationKeys, Maybe StoredIntrospection) SchemaCache
|
||||
_rscRebuild :: Inc.Rule (ReaderT BuildReason CacheBuild) (Metadata, ServerConfigCtx, InvalidationKeys, Maybe StoredIntrospection) SchemaCache
|
||||
}
|
||||
|
||||
bindErrorA ::
|
||||
|
@ -19,7 +19,7 @@ import Hasura.RQL.Types.Eventing
|
||||
import Hasura.RQL.Types.Source
|
||||
import Hasura.RQL.Types.Table (PrimaryKey)
|
||||
import Hasura.SQL.Backend
|
||||
import Hasura.Server.Types (HasServerConfigCtx, MaintenanceMode, ServerConfigCtx)
|
||||
import Hasura.Server.Types (MaintenanceMode, ServerConfigCtx)
|
||||
import Hasura.Session (UserInfo)
|
||||
import Hasura.Tracing qualified as Tracing
|
||||
|
||||
@ -187,7 +187,8 @@ class Backend b => BackendEventTrigger (b :: BackendType) where
|
||||
-- exist according to the event trigger's specification. If any SQL trigger doesn't
|
||||
-- exist then it will create it.
|
||||
createMissingSQLTriggers ::
|
||||
(MonadIO m, MonadError QErr m, MonadBaseControl IO m, Backend b, HasServerConfigCtx m) =>
|
||||
(MonadIO m, MonadError QErr m, MonadBaseControl IO m, Backend b) =>
|
||||
ServerConfigCtx ->
|
||||
SourceConfig b ->
|
||||
TableName b ->
|
||||
([ColumnInfo b], Maybe (PrimaryKey b (ColumnInfo b))) ->
|
||||
@ -332,7 +333,7 @@ instance BackendEventTrigger ('Postgres 'Citus) where
|
||||
redeliverEvent _ _ = throw400 NotSupported "Event triggers are not supported for Citus sources"
|
||||
unlockEventsInSource _ _ = runExceptT $ throw400 NotSupported "Event triggers are not supported for Citus sources"
|
||||
createTableEventTrigger _ _ _ _ _ _ _ _ = runExceptT $ throw400 NotSupported "Event triggers are not supported for Citus sources"
|
||||
createMissingSQLTriggers _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for Citus sources"
|
||||
createMissingSQLTriggers _ _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for Citus sources"
|
||||
checkIfTriggerExists _ _ _ = throw400 NotSupported $ "Event triggers are not supported for Citus sources"
|
||||
addCleanupSchedules _ _ = throw400 NotSupported $ "Event triggers are not supported for Citus sources"
|
||||
deleteAllScheduledCleanups _ _ = throw400 NotSupported $ "Event triggers are not supported for Citus sources"
|
||||
@ -401,7 +402,7 @@ instance BackendEventTrigger 'BigQuery where
|
||||
redeliverEvent _ _ = throw400 NotSupported "Event triggers are not supported for BigQuery sources"
|
||||
unlockEventsInSource _ _ = runExceptT $ throw400 NotSupported "Event triggers are not supported for BigQuery sources"
|
||||
createTableEventTrigger _ _ _ _ _ _ _ _ = runExceptT $ throw400 NotSupported "Event triggers are not supported for BigQuery sources"
|
||||
createMissingSQLTriggers _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for BigQuery sources"
|
||||
createMissingSQLTriggers _ _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for BigQuery sources"
|
||||
checkIfTriggerExists _ _ _ = throw400 NotSupported $ "Event triggers are not supported for BigQuery sources"
|
||||
addCleanupSchedules _ _ = throw400 NotSupported $ "Event triggers are not supported for BigQuery sources"
|
||||
deleteAllScheduledCleanups _ _ = throw400 NotSupported $ "Event triggers are not supported for BigQuery sources"
|
||||
@ -424,7 +425,7 @@ instance BackendEventTrigger 'MySQL where
|
||||
redeliverEvent _ _ = throw400 NotSupported "Event triggers are not supported for MySQL sources"
|
||||
unlockEventsInSource _ _ = runExceptT $ throw400 NotSupported "Event triggers are not supported for MySQL sources"
|
||||
createTableEventTrigger _ _ _ _ _ _ _ _ = runExceptT $ throw400 NotSupported "Event triggers are not supported for MySQL sources"
|
||||
createMissingSQLTriggers _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for MySQL sources"
|
||||
createMissingSQLTriggers _ _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for MySQL sources"
|
||||
checkIfTriggerExists _ _ _ = throw400 NotSupported $ "Event triggers are not supported for MySQL sources"
|
||||
addCleanupSchedules _ _ = throw400 NotSupported $ "Event triggers are not supported for MySQL sources"
|
||||
deleteAllScheduledCleanups _ _ = throw400 NotSupported $ "Event triggers are not supported for MySQL sources"
|
||||
@ -465,7 +466,7 @@ instance BackendEventTrigger 'DataConnector where
|
||||
runExceptT $ throw400 NotSupported "Event triggers are not supported for the Data Connector backend."
|
||||
createTableEventTrigger _ _ _ _ _ _ _ _ =
|
||||
runExceptT $ throw400 NotSupported "Event triggers are not supported for the Data Connector backend."
|
||||
createMissingSQLTriggers _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for Data Connector backend."
|
||||
createMissingSQLTriggers _ _ _ _ _ _ _ = throw400 NotSupported $ "Event triggers are not supported for Data Connector backend."
|
||||
checkIfTriggerExists _ _ _ = throw400 NotSupported $ "Event triggers are not supported for Data Connector backend."
|
||||
addCleanupSchedules _ _ = throw400 NotSupported $ "Event triggers are not supported for Data Connector backend."
|
||||
deleteAllScheduledCleanups _ _ = throw400 NotSupported $ "Event triggers are not supported for Data Connector backend."
|
||||
|
@ -167,8 +167,7 @@ withRecordInconsistency f = proc (e, (metadataObject, s)) -> do
|
||||
-- operations for triggering a schema cache rebuild
|
||||
|
||||
class (CacheRM m) => CacheRWM m where
|
||||
buildSchemaCacheWithOptions ::
|
||||
BuildReason -> CacheInvalidations -> Metadata -> m ()
|
||||
buildSchemaCacheWithOptions :: BuildReason -> CacheInvalidations -> Metadata -> m ()
|
||||
setMetadataResourceVersionInSchemaCache :: MetadataResourceVersion -> m ()
|
||||
|
||||
data BuildReason
|
||||
|
@ -186,6 +186,10 @@ instance (MonadResolveSource m) => MonadResolveSource (ReaderT r m) where
|
||||
getPGSourceResolver = lift getPGSourceResolver
|
||||
getMSSQLSourceResolver = lift getMSSQLSourceResolver
|
||||
|
||||
instance (MonadResolveSource m) => MonadResolveSource (StateT s m) where
|
||||
getPGSourceResolver = lift getPGSourceResolver
|
||||
getMSSQLSourceResolver = lift getMSSQLSourceResolver
|
||||
|
||||
instance (MonadResolveSource m) => MonadResolveSource (Tracing.TraceT m) where
|
||||
getPGSourceResolver = lift getPGSourceResolver
|
||||
getMSSQLSourceResolver = lift getMSSQLSourceResolver
|
||||
|
@ -388,15 +388,14 @@ runMetadataQuery ::
|
||||
MonadEventLogCleanup m,
|
||||
ProvidesHasuraServices m,
|
||||
MonadGetApiTimeLimit m,
|
||||
UserInfoM m,
|
||||
HasServerConfigCtx m
|
||||
UserInfoM m
|
||||
) =>
|
||||
AppContext ->
|
||||
RebuildableSchemaCache ->
|
||||
RQLMetadata ->
|
||||
m (EncJSON, RebuildableSchemaCache)
|
||||
runMetadataQuery appContext schemaCache RQLMetadata {..} = do
|
||||
AppEnv {..} <- askAppEnv
|
||||
appEnv@AppEnv {..} <- askAppEnv
|
||||
let logger = _lsLogger appEnvLoggers
|
||||
(metadata, currentResourceVersion) <- Tracing.newSpan "fetchMetadata" $ liftEitherM fetchMetadata
|
||||
let exportsMetadata = \case
|
||||
@ -422,12 +421,13 @@ runMetadataQuery appContext schemaCache RQLMetadata {..} = do
|
||||
if (exportsMetadata _rqlMetadata || queryModifiesMetadata _rqlMetadata)
|
||||
then emptyMetadataDefaults
|
||||
else acMetadataDefaults appContext
|
||||
serverConfigCtx = buildServerConfigCtx appEnv appContext
|
||||
((r, modMetadata), modSchemaCache, cacheInvalidations) <-
|
||||
runMetadataQueryM (acEnvironment appContext) currentResourceVersion _rqlMetadata
|
||||
-- TODO: remove this straight runReaderT that provides no actual new info
|
||||
& flip runReaderT logger
|
||||
& runMetadataT metadata metadataDefaults
|
||||
& runCacheRWT schemaCache
|
||||
& runCacheRWT serverConfigCtx schemaCache
|
||||
-- set modified metadata in storage
|
||||
if queryModifiesMetadata _rqlMetadata
|
||||
then case (appEnvEnableMaintenanceMode, appEnvEnableReadOnlyMode) of
|
||||
@ -458,7 +458,7 @@ runMetadataQuery appContext schemaCache RQLMetadata {..} = do
|
||||
(_, modSchemaCache', _) <-
|
||||
Tracing.newSpan "setMetadataResourceVersionInSchemaCache" $
|
||||
setMetadataResourceVersionInSchemaCache newResourceVersion
|
||||
& runCacheRWT modSchemaCache
|
||||
& runCacheRWT serverConfigCtx modSchemaCache
|
||||
|
||||
pure (r, modSchemaCache')
|
||||
(MaintenanceModeEnabled (), ReadOnlyModeDisabled) ->
|
||||
|
@ -186,15 +186,14 @@ runQuery ::
|
||||
MonadEventLogCleanup m,
|
||||
ProvidesHasuraServices m,
|
||||
MonadGetApiTimeLimit m,
|
||||
UserInfoM m,
|
||||
HasServerConfigCtx m
|
||||
UserInfoM m
|
||||
) =>
|
||||
AppContext ->
|
||||
RebuildableSchemaCache ->
|
||||
RQLQuery ->
|
||||
m (EncJSON, RebuildableSchemaCache)
|
||||
runQuery appContext sc query = do
|
||||
AppEnv {..} <- askAppEnv
|
||||
appEnv@AppEnv {..} <- askAppEnv
|
||||
let logger = _lsLogger appEnvLoggers
|
||||
when ((appEnvEnableReadOnlyMode == ReadOnlyModeEnabled) && queryModifiesUserDB query) $
|
||||
throw400 NotSupported "Cannot run write queries when read-only mode is enabled"
|
||||
@ -206,6 +205,7 @@ runQuery appContext sc query = do
|
||||
if (exportsMetadata query)
|
||||
then emptyMetadataDefaults
|
||||
else acMetadataDefaults appContext
|
||||
serverConfigCtx = buildServerConfigCtx appEnv appContext
|
||||
|
||||
(metadata, currentResourceVersion) <- liftEitherM fetchMetadata
|
||||
((result, updatedMetadata), updatedCache, invalidations) <-
|
||||
@ -213,7 +213,7 @@ runQuery appContext sc query = do
|
||||
-- TODO: remove this straight runReaderT that provides no actual new info
|
||||
& flip runReaderT logger
|
||||
& runMetadataT metadata metadataDefaults
|
||||
& runCacheRWT sc
|
||||
& runCacheRWT serverConfigCtx sc
|
||||
when (queryModifiesSchemaCache query) $ do
|
||||
case appEnvEnableMaintenanceMode of
|
||||
MaintenanceModeDisabled -> do
|
||||
|
@ -111,24 +111,24 @@ runQuery ::
|
||||
MonadResolveSource m,
|
||||
MonadQueryTags m,
|
||||
ProvidesHasuraServices m,
|
||||
UserInfoM m,
|
||||
HasServerConfigCtx m
|
||||
UserInfoM m
|
||||
) =>
|
||||
AppContext ->
|
||||
RebuildableSchemaCache ->
|
||||
RQLQuery ->
|
||||
m (EncJSON, RebuildableSchemaCache)
|
||||
runQuery appContext schemaCache rqlQuery = do
|
||||
AppEnv {..} <- askAppEnv
|
||||
appEnv@AppEnv {..} <- askAppEnv
|
||||
when ((appEnvEnableReadOnlyMode == ReadOnlyModeEnabled) && queryModifiesUserDB rqlQuery) $
|
||||
throw400 NotSupported "Cannot run write queries when read-only mode is enabled"
|
||||
|
||||
let serverConfigCtx = buildServerConfigCtx appEnv appContext
|
||||
(metadata, currentResourceVersion) <- Tracing.newSpan "fetchMetadata" $ liftEitherM fetchMetadata
|
||||
((result, updatedMetadata), updatedCache, invalidations) <-
|
||||
runQueryM (acEnvironment appContext) rqlQuery
|
||||
-- We can use defaults here unconditionally, since there is no MD export function in V2Query
|
||||
& runMetadataT metadata (acMetadataDefaults appContext)
|
||||
& runCacheRWT schemaCache
|
||||
& runCacheRWT serverConfigCtx schemaCache
|
||||
when (queryModifiesSchema rqlQuery) $ do
|
||||
case appEnvEnableMaintenanceMode of
|
||||
MaintenanceModeDisabled -> do
|
||||
|
@ -156,25 +156,6 @@ instance MonadTrans Handler where
|
||||
instance Monad m => UserInfoM (Handler m) where
|
||||
askUserInfo = asks hcUser
|
||||
|
||||
instance (HasAppEnv m) => HasServerConfigCtx (Handler m) where
|
||||
askServerConfigCtx = Handler do
|
||||
AppEnv {..} <- askAppEnv
|
||||
AppContext {..} <- asks hcAppContext
|
||||
pure
|
||||
ServerConfigCtx
|
||||
{ _sccFunctionPermsCtx = acFunctionPermsCtx,
|
||||
_sccRemoteSchemaPermsCtx = acRemoteSchemaPermsCtx,
|
||||
_sccSQLGenCtx = acSQLGenCtx,
|
||||
_sccMaintenanceMode = appEnvEnableMaintenanceMode,
|
||||
_sccExperimentalFeatures = acExperimentalFeatures,
|
||||
_sccEventingMode = appEnvEventingMode,
|
||||
_sccReadOnlyMode = appEnvEnableReadOnlyMode,
|
||||
_sccDefaultNamingConvention = acDefaultNamingConvention,
|
||||
_sccMetadataDefaults = acMetadataDefaults,
|
||||
_sccCheckFeatureFlag = appEnvCheckFeatureFlag,
|
||||
_sccApolloFederationStatus = acApolloFederationStatus
|
||||
}
|
||||
|
||||
runHandler :: (HasResourceLimits m, MonadBaseControl IO m) => HandlerCtx -> Handler m a -> m (Either QErr a)
|
||||
runHandler ctx (Handler r) = do
|
||||
handlerLimit <- askHTTPHandlerLimit
|
||||
@ -448,8 +429,7 @@ v1QueryHandler ::
|
||||
MonadEventLogCleanup m,
|
||||
ProvidesNetwork m,
|
||||
MonadGetApiTimeLimit m,
|
||||
UserInfoM m,
|
||||
HasServerConfigCtx m
|
||||
UserInfoM m
|
||||
) =>
|
||||
(m (EncJSON, RebuildableSchemaCache) -> m EncJSON) ->
|
||||
RQLQuery ->
|
||||
@ -481,8 +461,7 @@ v1MetadataHandler ::
|
||||
HasAppEnv m,
|
||||
ProvidesNetwork m,
|
||||
MonadGetApiTimeLimit m,
|
||||
UserInfoM m,
|
||||
HasServerConfigCtx m
|
||||
UserInfoM m
|
||||
) =>
|
||||
(m (EncJSON, RebuildableSchemaCache) -> m EncJSON) ->
|
||||
RQLMetadata ->
|
||||
@ -511,8 +490,7 @@ v2QueryHandler ::
|
||||
HasAppEnv m,
|
||||
EB.MonadQueryTags m,
|
||||
ProvidesNetwork m,
|
||||
UserInfoM m,
|
||||
HasServerConfigCtx m
|
||||
UserInfoM m
|
||||
) =>
|
||||
(m (EncJSON, RebuildableSchemaCache) -> m EncJSON) ->
|
||||
V2Q.RQLQuery ->
|
||||
|
@ -42,7 +42,6 @@ import Hasura.Server.AppStateRef
|
||||
import Hasura.Server.Logging
|
||||
import Hasura.Server.Types
|
||||
import Hasura.Services
|
||||
import Hasura.Session
|
||||
import Refined (NonNegative, Refined, unrefine)
|
||||
|
||||
data ThreadError
|
||||
@ -260,44 +259,6 @@ processor
|
||||
metaVersion <- liftIO $ STM.atomically $ STM.takeTMVar metaVersionRef
|
||||
refreshSchemaCache metaVersion appStateRef TTProcessor logTVar
|
||||
|
||||
newtype SchemaUpdateT m a = SchemaUpdateT (AppContext -> m a)
|
||||
deriving
|
||||
( Functor,
|
||||
Applicative,
|
||||
Monad,
|
||||
MonadError e,
|
||||
MonadIO,
|
||||
MonadMetadataStorage,
|
||||
ProvidesNetwork,
|
||||
MonadResolveSource
|
||||
)
|
||||
via (ReaderT AppContext m)
|
||||
deriving (MonadTrans) via (ReaderT AppContext)
|
||||
|
||||
runSchemaUpdate :: AppContext -> SchemaUpdateT m a -> m a
|
||||
runSchemaUpdate appContext (SchemaUpdateT action) = action appContext
|
||||
|
||||
instance (Monad m) => UserInfoM (SchemaUpdateT m) where
|
||||
askUserInfo = pure adminUserInfo
|
||||
|
||||
instance (HasAppEnv m) => HasServerConfigCtx (SchemaUpdateT m) where
|
||||
askServerConfigCtx = SchemaUpdateT \AppContext {..} -> do
|
||||
AppEnv {..} <- askAppEnv
|
||||
pure
|
||||
ServerConfigCtx
|
||||
{ _sccFunctionPermsCtx = acFunctionPermsCtx,
|
||||
_sccRemoteSchemaPermsCtx = acRemoteSchemaPermsCtx,
|
||||
_sccSQLGenCtx = acSQLGenCtx,
|
||||
_sccMaintenanceMode = appEnvEnableMaintenanceMode,
|
||||
_sccExperimentalFeatures = acExperimentalFeatures,
|
||||
_sccEventingMode = appEnvEventingMode,
|
||||
_sccReadOnlyMode = appEnvEnableReadOnlyMode,
|
||||
_sccDefaultNamingConvention = acDefaultNamingConvention,
|
||||
_sccMetadataDefaults = acMetadataDefaults,
|
||||
_sccCheckFeatureFlag = appEnvCheckFeatureFlag,
|
||||
_sccApolloFederationStatus = acApolloFederationStatus
|
||||
}
|
||||
|
||||
refreshSchemaCache ::
|
||||
( MonadIO m,
|
||||
MonadBaseControl IO m,
|
||||
@ -316,14 +277,15 @@ refreshSchemaCache
|
||||
appStateRef
|
||||
threadType
|
||||
logTVar = do
|
||||
AppEnv {..} <- askAppEnv
|
||||
appEnv@AppEnv {..} <- askAppEnv
|
||||
let logger = _lsLogger appEnvLoggers
|
||||
respErr <- runExceptT $
|
||||
withSchemaCacheUpdate appStateRef logger (Just logTVar) $ do
|
||||
rebuildableCache <- liftIO $ fst <$> readSchemaCacheRef appStateRef
|
||||
appContext <- liftIO $ getAppContext appStateRef
|
||||
(msg, cache, _) <- runSchemaUpdate appContext $
|
||||
runCacheRWT rebuildableCache $ do
|
||||
let serverConfigCtx = buildServerConfigCtx appEnv appContext
|
||||
(msg, cache, _) <-
|
||||
runCacheRWT serverConfigCtx rebuildableCache $ do
|
||||
schemaCache <- askSchemaCache
|
||||
case scMetadataResourceVersion schemaCache of
|
||||
-- While starting up, the metadata resource version is set to nothing, so we want to set the version
|
||||
|
@ -141,6 +141,15 @@ data ReadOnlyMode = ReadOnlyModeEnabled | ReadOnlyModeDisabled
|
||||
data EventingMode = EventingEnabled | EventingDisabled
|
||||
deriving (Show, Eq)
|
||||
|
||||
-- | This type represents an aggregate of different configuration options used
|
||||
-- throughout the engine. The fields are the union of a subset of 'AppEnv' and a
|
||||
-- subset of 'AppContext'.
|
||||
--
|
||||
-- This type should be considered as deprecated: avoid using it directly when
|
||||
-- you can use 'AppEnv' or 'AppContext', and avoid using the entirety of it when
|
||||
-- you only need a subset of the fields. Also avoid adding new fields if
|
||||
-- possible, but if you do so, make sure to adjust the @Eq@ instance
|
||||
-- accordingly.
|
||||
data ServerConfigCtx = ServerConfigCtx
|
||||
{ _sccFunctionPermsCtx :: Options.InferFunctionPermissions,
|
||||
_sccRemoteSchemaPermsCtx :: Options.RemoteSchemaPermissions,
|
||||
@ -156,6 +165,28 @@ data ServerConfigCtx = ServerConfigCtx
|
||||
_sccApolloFederationStatus :: ApolloFederationStatus
|
||||
}
|
||||
|
||||
-- We are currently using the entire 'ServerConfigCtx' as an input to the schema
|
||||
-- cache build, and it therefore requires an 'Eq' instance. However, only a few
|
||||
-- fields will change over time: those coming from the 'AppContext', and not
|
||||
-- those coming from the 'AppEnv'. Consequently, this instance only checks the
|
||||
-- relevant fields.
|
||||
--
|
||||
-- The way to fix this will be to use a smaller type as the input to the schema
|
||||
-- build, such as 'AppContext' (or, rather, a relevant subset), on which a
|
||||
-- "correct" @Eq@ instance can be defined.
|
||||
instance Eq ServerConfigCtx where
|
||||
(==) = (==) `on` extractDynamicFields
|
||||
where
|
||||
extractDynamicFields ServerConfigCtx {..} =
|
||||
( _sccFunctionPermsCtx,
|
||||
_sccRemoteSchemaPermsCtx,
|
||||
_sccSQLGenCtx,
|
||||
_sccExperimentalFeatures,
|
||||
_sccDefaultNamingConvention,
|
||||
_sccMetadataDefaults,
|
||||
_sccApolloFederationStatus
|
||||
)
|
||||
|
||||
askMetadataDefaults :: HasServerConfigCtx m => m MetadataDefaults
|
||||
askMetadataDefaults = do
|
||||
ServerConfigCtx {_sccMetadataDefaults} <- askServerConfigCtx
|
||||
|
@ -119,7 +119,7 @@ main = do
|
||||
emptyMetadataDefaults
|
||||
(CheckFeatureFlag $ FF.checkFeatureFlag mempty)
|
||||
ApolloFederationDisabled
|
||||
cacheBuildParams = CacheBuildParams httpManager (mkPgSourceResolver print) mkMSSQLSourceResolver serverConfigCtx
|
||||
cacheBuildParams = CacheBuildParams httpManager (mkPgSourceResolver print) mkMSSQLSourceResolver
|
||||
|
||||
(_appInit, appEnv) <-
|
||||
lowerManagedT $
|
||||
@ -144,7 +144,7 @@ main = do
|
||||
snd
|
||||
<$> (liftEitherM . runExceptT . _pecRunTx pgContext (PGExecCtxInfo (Tx PG.ReadWrite Nothing) InternalRawQuery))
|
||||
(migrateCatalog (Just sourceConfig) defaultPostgresExtensionsSchema maintenanceMode =<< liftIO getCurrentTime)
|
||||
schemaCache <- runCacheBuild cacheBuildParams $ buildRebuildableSchemaCache logger envMap metadata
|
||||
schemaCache <- runCacheBuild cacheBuildParams $ buildRebuildableSchemaCache logger envMap metadata serverConfigCtx
|
||||
pure (metadata, schemaCache)
|
||||
|
||||
cacheRef <- newMVar schemaCache
|
||||
|
@ -86,15 +86,15 @@ instance
|
||||
CacheRWM (CacheRefT m)
|
||||
where
|
||||
buildSchemaCacheWithOptions reason invalidations metadata = do
|
||||
scVar <- asks snd
|
||||
(serverConfigCtx, scVar) <- ask
|
||||
modifyMVar scVar \schemaCache -> do
|
||||
((), cache, _) <- runCacheRWT schemaCache (buildSchemaCacheWithOptions reason invalidations metadata)
|
||||
((), cache, _) <- runCacheRWT serverConfigCtx schemaCache (buildSchemaCacheWithOptions reason invalidations metadata)
|
||||
pure (cache, ())
|
||||
|
||||
setMetadataResourceVersionInSchemaCache resourceVersion = do
|
||||
scVar <- asks snd
|
||||
(serverConfigCtx, scVar) <- ask
|
||||
modifyMVar scVar \schemaCache -> do
|
||||
((), cache, _) <- runCacheRWT schemaCache (setMetadataResourceVersionInSchemaCache resourceVersion)
|
||||
((), cache, _) <- runCacheRWT serverConfigCtx schemaCache (setMetadataResourceVersionInSchemaCache resourceVersion)
|
||||
pure (cache, ())
|
||||
|
||||
instance Example (MetadataT (CacheRefT m) ()) where
|
||||
@ -128,8 +128,9 @@ suite srcConfig pgExecCtx pgConnInfo = do
|
||||
liftIO $ putStrLn $ LBS.toString $ encode $ EngineLog t logLevel logType logDetail
|
||||
|
||||
migrateCatalogAndBuildCache env time = do
|
||||
serverConfigCtx <- askServerConfigCtx
|
||||
(migrationResult, metadata) <- runTx' pgExecCtx $ migrateCatalog (Just srcConfig) (ExtensionsSchema "public") MaintenanceModeDisabled time
|
||||
(,migrationResult) <$> runCacheBuildM (buildRebuildableSchemaCache logger env metadata)
|
||||
(,migrationResult) <$> runCacheBuildM (buildRebuildableSchemaCache logger env metadata serverConfigCtx)
|
||||
|
||||
dropAndInit env time = lift do
|
||||
scVar <- asks snd
|
||||
|
Loading…
Reference in New Issue
Block a user