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:
Antoine Leblanc 2023-03-27 18:42:37 +01:00 committed by hasura-bot
parent 2d9c8299c2
commit 7aa341944b
18 changed files with 190 additions and 181 deletions

View File

@ -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

View File

@ -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
}

View File

@ -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)

View File

@ -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

View File

@ -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

View File

@ -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 dont 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 dont 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

View File

@ -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 ::

View File

@ -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."

View File

@ -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

View File

@ -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

View File

@ -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) ->

View File

@ -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

View File

@ -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

View File

@ -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 ->

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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