Build the first schema cache in the app monad.

### Description

This PR is one more step towards #8344: it splits `initializeContext` into two parts: the first one builds the `AppEnv`, and the second builds the `AppContext` and its corresponding `AppStateRef`. Splitting it allows us to run the first schema cache build _in the app monad_, which was one of the steps required to remove `CacheBuild`, and ultimately to achieve #8344, which in turn is a blocker for several projects.

PR-URL: https://github.com/hasura/graphql-engine-mono/pull/8445
GitOrigin-RevId: 0ef9eeec0c941b915da505cc9b1c60f36b108a56
This commit is contained in:
Antoine Leblanc 2023-03-27 11:25:55 +01:00 committed by hasura-bot
parent ce93288b94
commit dd635fbbc7
4 changed files with 319 additions and 263 deletions

View File

@ -402,9 +402,10 @@ runApp serveOptions = do
liftIO $ createServerMetrics $ EKG.subset ServerSubset store
pure (EKG.subset EKG.emptyOf store, serverMetrics)
prometheusMetrics <- makeDummyPrometheusMetrics
let managedServerCtx = App.initialiseContext env defaultConnInfo serveOptions Nothing serverMetrics prometheusMetrics sampleAlways
runManagedT managedServerCtx \(appCtx, appEnv) ->
App.runAppM appEnv $
let managedServerCtx = App.initialiseAppEnv env defaultConnInfo serveOptions Nothing serverMetrics prometheusMetrics sampleAlways
runManagedT managedServerCtx \(appInit, appEnv) ->
App.runAppM appEnv do
appCtx <- App.initialiseAppContext env serveOptions appInit
lowerManagedT $
App.runHGEServer
(const $ pure ())

View File

@ -92,19 +92,17 @@ runApp env (HGEOptions rci metadataDbUrl hgeCmd) = do
prometheusMetrics <- makeDummyPrometheusMetrics
-- It'd be nice if we didn't have to call runManagedT twice here, but
-- there is a data dependency problem since the call to runPGMetadataStorageApp
-- below depends on appCtx.
runManagedT (initialiseContext env basicConnectionInfo serveOptions Nothing serverMetrics prometheusMetrics sampleAlways) $ \(appStateRef, appEnv) -> do
-- It'd be nice if we didn't have to call lowerManagedT twice here, but
-- there is a data dependency problem since the call to runAppM below
-- depends on appCtx.
runManagedT (initialiseAppEnv env basicConnectionInfo serveOptions Nothing serverMetrics prometheusMetrics sampleAlways) \(appInit, appEnv) -> do
-- Catches the SIGTERM signal and initiates a graceful shutdown.
-- Graceful shutdown for regular HTTP requests is already implemented in
-- Warp, and is triggered by invoking the 'closeSocket' callback.
-- We only catch the SIGTERM signal once, that is, if the user hits CTRL-C
-- once again, we terminate the process immediately.
liftIO $ do
void $ Signals.installHandler Signals.sigTERM (Signals.CatchOnce (shutdownGracefully $ appEnvShutdownLatch appEnv)) Nothing
void $ Signals.installHandler Signals.sigINT (Signals.CatchOnce (shutdownGracefully $ appEnvShutdownLatch appEnv)) Nothing
void $ Signals.installHandler Signals.sigTERM (Signals.CatchOnce (shutdownGracefully $ appEnvShutdownLatch appEnv)) Nothing
void $ Signals.installHandler Signals.sigINT (Signals.CatchOnce (shutdownGracefully $ appEnvShutdownLatch appEnv)) Nothing
let Loggers _ logger _ = appEnvLoggers appEnv
@ -112,7 +110,8 @@ runApp env (HGEOptions rci metadataDbUrl hgeCmd) = do
C.forkImmortal "ourIdleGC" logger $
GC.ourIdleGC logger (seconds 0.3) (seconds 10) (seconds 60)
runAppM appEnv $
runAppM appEnv do
appStateRef <- initialiseAppContext env serveOptions appInit
lowerManagedT $
runHGEServer (const $ pure ()) appStateRef initTime Nothing ekgStore
HCExport -> do

View File

@ -22,6 +22,7 @@ module Hasura.App
-- * logging
mkLoggers,
mkPGLogger,
flushLogger,
-- * basic connection info
BasicConnectionInfo (..),
@ -29,20 +30,21 @@ module Hasura.App
initBasicConnectionInfo,
resolvePostgresConnInfo,
-- * app init
initialiseAppEnv,
initialiseAppContext,
migrateCatalogAndFetchMetadata,
buildFirstSchemaCache,
initSubscriptionsState,
initLockedEventsCtx,
-- * app monad
AppM,
runAppM,
-- * misc
flushLogger,
getCatalogStateTx,
updateJwkCtxThread,
initialiseContext,
initSubscriptionsState,
initLockedEventsCtx,
initSQLGenCtx,
migrateCatalogAndFetchMetadata,
buildFirstSchemaCache,
notifySchemaCacheSyncTx,
parseArgs,
runHGEServer,
@ -214,6 +216,7 @@ printJSON = liftIO . BLC.putStrLn . A.encode
mkPGLogger :: Logger Hasura -> PG.PGLogger
mkPGLogger (Logger logger) (PG.PLERetryMsg msg) = logger $ PGLog LevelWarn msg
-- | Create all loggers based on the set of enabled logs and chosen log level.
mkLoggers ::
(MonadIO m, MonadBaseControl IO m) =>
HashSet (EngineLogType Hasura) ->
@ -225,6 +228,13 @@ mkLoggers enabledLogs logLevel = do
pgLogger = mkPGLogger logger
pure $ Loggers loggerCtx logger pgLogger
-- | If an exception is thrown, and the process exits without the log buffer
-- being flushed, we will be missing some log lines (see:
-- https://github.com/hasura/graphql-engine/issues/4772). This function forces a
-- flush of the buffer.
flushLogger :: MonadIO m => LoggerCtx impl -> m ()
flushLogger = liftIO . FL.flushLogStr . _lcLoggerSet
--------------------------------------------------------------------------------
-- Basic connection info
@ -346,6 +356,289 @@ resolvePostgresConnInfo env dbUrlConf (fromMaybe 1 -> retries) = do
liftIO (throwErrJExit InvalidDatabaseConnectionParamsError err)
pure $ PG.ConnInfo retries $ PG.CDDatabaseURI $ txtToBs dbUrlText
--------------------------------------------------------------------------------
-- App init
-- | The initialisation of the app is split into several functions, for clarity;
-- but there are several pieces of information that need to be threaded across
-- those initialisation functions. This small data structure groups together all
-- such pieces of information that are required throughout the initialisation,
-- but that aren't needed in the rest of the application.
data AppInit = AppInit
{ aiTLSAllowListRef :: TLSAllowListRef Hasura,
aiMetadata :: Metadata
}
-- | Initializes or migrates the catalog and creates the 'AppEnv' required to
-- start the server, and also create the 'AppInit' that needs to be threaded
-- along the init code.
--
-- For historical reasons, this function performs a few additional startup tasks
-- that are not required to create the 'AppEnv', such as starting background
-- processes and logging startup information. All of those are flagged with a
-- comment marking them as a side-effect.
initialiseAppEnv ::
(C.ForkableMonadIO m, MonadCatch m) =>
Env.Environment ->
BasicConnectionInfo ->
ServeOptions Hasura ->
Maybe ES.SubscriptionPostPollHook ->
ServerMetrics ->
PrometheusMetrics ->
SamplingPolicy ->
ManagedT m (AppInit, AppEnv)
initialiseAppEnv env BasicConnectionInfo {..} serveOptions@ServeOptions {..} liveQueryHook serverMetrics prometheusMetrics traceSamplingPolicy = do
loggers@(Loggers loggerCtx logger pgLogger) <- mkLoggers soEnabledLogTypes soLogLevel
-- SIDE EFFECT: print a warning if no admin secret is set.
when (null soAdminSecret) $
unLogger
logger
StartupLog
{ slLogLevel = LevelWarn,
slKind = "no_admin_secret",
slInfo = A.toJSON ("WARNING: No admin secret provided" :: Text)
}
-- SIDE EFFECT: log all server options.
unLogger logger $ serveOptsToLog serveOptions
-- SIDE EFFECT: log metadata postgres connection info.
unLogger logger $ connInfoToLog bciMetadataConnInfo
-- Generate the instance id.
instanceId <- liftIO generateInstanceId
-- Init metadata db pool.
metadataDbPool <-
allocate
(liftIO $ PG.initPGPool bciMetadataConnInfo soConnParams pgLogger)
(liftIO . PG.destroyPGPool)
-- Migrate the catalog and fetch the metdata.
metadata <-
lift $
flip onException (flushLogger loggerCtx) $
migrateCatalogAndFetchMetadata
logger
metadataDbPool
bciDefaultPostgres
soEnableMaintenanceMode
soExtensionsSchema
-- Create the TLSAllowListRef and the HTTP Manager.
tlsAllowListRef <- liftIO $ createTLSAllowListRef $ networkTlsAllowlist $ _metaNetwork metadata
httpManager <- liftIO $ mkHttpManager (readTLSAllowList tlsAllowListRef) mempty
-- Start a background thread for listening schema sync events from other
-- server instances (an interval of 0 indicates that no schema sync is
-- required). Logs whether the thread is started or not, and with what
-- interval.
-- TODO: extract into a separate init function.
metaVersionRef <- liftIO $ STM.newEmptyTMVarIO
case soSchemaPollInterval of
Skip -> unLogger logger $ mkGenericLog @Text LevelInfo "schema-sync" "Schema sync disabled"
Interval interval -> do
unLogger logger $ mkGenericLog @String LevelInfo "schema-sync" ("Schema sync enabled. Polling at " <> show interval)
void $ startSchemaSyncListenerThread logger metadataDbPool instanceId interval metaVersionRef
-- Generate the shutdown latch.
latch <- liftIO newShutdownLatch
-- Generate subscription state.
subscriptionsState <- liftIO $ initSubscriptionsState logger liveQueryHook
-- Generate event's trigger shared state
lockedEventsCtx <- liftIO $ initLockedEventsCtx
pure
( AppInit
{ aiTLSAllowListRef = tlsAllowListRef,
aiMetadata = metadata
},
AppEnv
{ appEnvPort = soPort,
appEnvHost = soHost,
appEnvMetadataDbPool = metadataDbPool,
appEnvManager = httpManager,
appEnvLoggers = loggers,
appEnvMetadataVersionRef = metaVersionRef,
appEnvInstanceId = instanceId,
appEnvEnableMaintenanceMode = soEnableMaintenanceMode,
appEnvLoggingSettings = LoggingSettings soEnabledLogTypes soEnableMetadataQueryLogging,
appEnvEventingMode = soEventingMode,
appEnvEnableReadOnlyMode = soReadOnlyMode,
appEnvServerMetrics = serverMetrics,
appEnvShutdownLatch = latch,
appEnvMetaVersionRef = metaVersionRef,
appEnvPrometheusMetrics = prometheusMetrics,
appEnvTraceSamplingPolicy = traceSamplingPolicy,
appEnvSubscriptionState = subscriptionsState,
appEnvLockedEventsCtx = lockedEventsCtx,
appEnvConnParams = soConnParams,
appEnvTxIso = soTxIso,
appEnvConsoleAssetsDir = soConsoleAssetsDir,
appEnvConsoleSentryDsn = soConsoleSentryDsn,
appEnvConnectionOptions = soConnectionOptions,
appEnvWebSocketKeepAlive = soWebSocketKeepAlive,
appEnvWebSocketConnectionInitTimeout = soWebSocketConnectionInitTimeout,
appEnvGracefulShutdownTimeout = soGracefulShutdownTimeout,
appEnvCheckFeatureFlag = CheckFeatureFlag $ checkFeatureFlag env,
appEnvSchemaPollInterval = soSchemaPollInterval
}
)
-- | Initializes the 'AppContext' and returns a corresponding 'AppStateRef'.
--
-- This function is meant to be run in the app monad, which provides the
-- 'AppEnv'.
initialiseAppContext ::
(C.ForkableMonadIO m, HasAppEnv m) =>
Env.Environment ->
ServeOptions Hasura ->
AppInit ->
m (AppStateRef Hasura)
initialiseAppContext env serveOptions@ServeOptions {..} AppInit {..} = do
AppEnv {..} <- askAppEnv
let Loggers _ logger pgLogger = appEnvLoggers
sqlGenCtx = initSQLGenCtx soExperimentalFeatures soStringifyNum soDangerousBooleanCollapse
serverConfigCtx =
ServerConfigCtx
soInferFunctionPermissions
soEnableRemoteSchemaPermissions
sqlGenCtx
soEnableMaintenanceMode
soExperimentalFeatures
soEventingMode
soReadOnlyMode
soDefaultNamingConvention
soMetadataDefaults
(CheckFeatureFlag $ checkFeatureFlag env)
soApolloFederationStatus
-- Create the schema cache
rebuildableSchemaCache <-
buildFirstSchemaCache
env
logger
serverConfigCtx
(mkPgSourceResolver pgLogger)
mkMSSQLSourceResolver
aiMetadata
appEnvManager
-- Build the RebuildableAppContext.
-- (See note [Hasura Application State].)
rebuildableAppCtxE <- liftIO $ runExceptT (buildRebuildableAppContext (logger, appEnvManager) serveOptions env)
!rebuildableAppCtx <- onLeft rebuildableAppCtxE $ \e -> throwErrExit InvalidEnvironmentVariableOptionsError $ T.unpack $ qeError e
-- Initialise the 'AppStateRef' from 'RebuildableSchemaCacheRef' and 'RebuildableAppContext'.
initialiseAppStateRef aiTLSAllowListRef appEnvServerMetrics rebuildableSchemaCache rebuildableAppCtx
-- | Runs catalogue migration, and returns the metadata that was fetched.
--
-- On success, this function logs the result of the migration, on failure it
-- logs a 'catalog_migrate' error and throws a fatal error.
migrateCatalogAndFetchMetadata ::
(MonadIO m, MonadBaseControl IO m) =>
Logger Hasura ->
PG.PGPool ->
Maybe (SourceConnConfiguration ('Postgres 'Vanilla)) ->
MaintenanceMode () ->
ExtensionsSchema ->
m Metadata
migrateCatalogAndFetchMetadata
logger
pool
defaultSourceConfig
maintenanceMode
extensionsSchema = do
-- TODO: should we allow the migration to happen during maintenance mode?
-- Allowing this can be a sanity check, to see if the hdb_catalog in the
-- DB has been set correctly
currentTime <- liftIO Clock.getCurrentTime
result <-
runExceptT $
PG.runTx pool (PG.Serializable, Just PG.ReadWrite) $
migrateCatalog
defaultSourceConfig
extensionsSchema
maintenanceMode
currentTime
case result of
Left err -> do
unLogger
logger
StartupLog
{ slLogLevel = LevelError,
slKind = "catalog_migrate",
slInfo = A.toJSON err
}
liftIO (throwErrJExit DatabaseMigrationError err)
Right (migrationResult, metadata) -> do
unLogger logger migrationResult
pure metadata
-- | Build the original 'RebuildableSchemaCache'.
--
-- On error, it logs a 'catalog_migrate' error and throws a fatal error. This
-- misnomer is intentional: it is to preserve a previous behaviour of the code
-- and avoid a breaking change.
buildFirstSchemaCache ::
(MonadIO m) =>
Env.Environment ->
Logger Hasura ->
ServerConfigCtx ->
SourceResolver ('Postgres 'Vanilla) ->
SourceResolver ('MSSQL) ->
Metadata ->
HTTP.Manager ->
m RebuildableSchemaCache
buildFirstSchemaCache
env
logger
serverConfigCtx
pgSourceResolver
mssqlSourceResolver
metadata
httpManager = do
let cacheBuildParams = CacheBuildParams httpManager pgSourceResolver mssqlSourceResolver serverConfigCtx
buildReason = CatalogSync
result <-
runExceptT $
runCacheBuild cacheBuildParams $
buildRebuildableSchemaCacheWithReason buildReason logger env metadata
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
-- error in the first schema cache build would be reported as
-- follows. Changing this will be a breaking change.
unLogger
logger
StartupLog
{ slLogLevel = LevelError,
slKind = "catalog_migrate",
slInfo = A.toJSON err
}
liftIO (throwErrJExit DatabaseMigrationError err)
initSubscriptionsState ::
Logger Hasura ->
Maybe ES.SubscriptionPostPollHook ->
IO ES.SubscriptionsState
initSubscriptionsState logger liveQueryHook = ES.initSubscriptionsState postPollHook
where
postPollHook = fromMaybe (ES.defaultSubscriptionPostPollHook logger) liveQueryHook
initLockedEventsCtx :: IO LockedEventsCtx
initLockedEventsCtx =
liftM4
LockedEventsCtx
(STM.newTVarIO mempty)
(STM.newTVarIO mempty)
(STM.newTVarIO mempty)
(STM.newTVarIO mempty)
--------------------------------------------------------------------------------
-- App monad
@ -513,6 +806,8 @@ instance MonadMetadataStorage AppM where
instance MonadMetadataStorageQueryAPI AppM
--------------------------------------------------------------------------------
-- misc
-- TODO(SOLOMON): Move Into `Hasura.Server.Init`. Unable to do so
-- currently due `throwErrExit`.
@ -544,237 +839,6 @@ updateJwkCtxThread getAppCtx httpManager logger = forever $ do
updateJwkCtx authMode httpManager logger
liftIO $ sleep $ seconds 1
initSubscriptionsState ::
Logger Hasura ->
Maybe ES.SubscriptionPostPollHook ->
IO ES.SubscriptionsState
initSubscriptionsState logger liveQueryHook = ES.initSubscriptionsState postPollHook
where
postPollHook = fromMaybe (ES.defaultSubscriptionPostPollHook logger) liveQueryHook
initLockedEventsCtx :: IO LockedEventsCtx
initLockedEventsCtx = LockedEventsCtx <$> STM.newTVarIO mempty <*> STM.newTVarIO mempty <*> STM.newTVarIO mempty <*> STM.newTVarIO mempty
-- | Initializes or migrates the catalog and returns the context required to start the server.
initialiseContext ::
(C.ForkableMonadIO m, MonadCatch m) =>
Env.Environment ->
BasicConnectionInfo ->
ServeOptions Hasura ->
Maybe ES.SubscriptionPostPollHook ->
ServerMetrics ->
PrometheusMetrics ->
SamplingPolicy ->
ManagedT m (AppStateRef Hasura, AppEnv)
initialiseContext env BasicConnectionInfo {..} serveOptions@ServeOptions {..} liveQueryHook serverMetrics prometheusMetrics traceSamplingPolicy = do
instanceId <- liftIO generateInstanceId
latch <- liftIO newShutdownLatch
loggers@(Loggers loggerCtx logger pgLogger) <- mkLoggers soEnabledLogTypes soLogLevel
when (null soAdminSecret) $ do
let errMsg :: Text
errMsg = "WARNING: No admin secret provided"
unLogger logger $
StartupLog
{ slLogLevel = LevelWarn,
slKind = "no_admin_secret",
slInfo = A.toJSON errMsg
}
-- log serve options
unLogger logger $ serveOptsToLog serveOptions
-- log metadata connection info
unLogger logger $ connInfoToLog bciMetadataConnInfo
metadataDbPool <-
allocate
(liftIO $ PG.initPGPool bciMetadataConnInfo soConnParams pgLogger)
(liftIO . PG.destroyPGPool)
let sqlGenCtx = initSQLGenCtx soExperimentalFeatures soStringifyNum soDangerousBooleanCollapse
checkFeatureFlag' = CheckFeatureFlag $ checkFeatureFlag env
serverConfigCtx =
ServerConfigCtx
soInferFunctionPermissions
soEnableRemoteSchemaPermissions
sqlGenCtx
soEnableMaintenanceMode
soExperimentalFeatures
soEventingMode
soReadOnlyMode
soDefaultNamingConvention
soMetadataDefaults
checkFeatureFlag'
soApolloFederationStatus
-- Migrate the catalog and fetch the metdata
metadata <-
lift $
flip onException (flushLogger loggerCtx) $
migrateCatalogAndFetchMetadata
logger
metadataDbPool
bciDefaultPostgres
soEnableMaintenanceMode
soExtensionsSchema
-- Create the TLSAllowListRef, and the HTTP Manager
tlsAllowListRef <- liftIO $ createTLSAllowListRef $ networkTlsAllowlist $ _metaNetwork metadata
httpManager <- liftIO $ mkHttpManager (readTLSAllowList tlsAllowListRef) mempty
-- Create the schema cache
rebuildableSchemaCache <-
lift $
flip onException (flushLogger $ _lsLoggerCtx loggers) $
buildFirstSchemaCache
env
logger
serverConfigCtx
(mkPgSourceResolver pgLogger)
mkMSSQLSourceResolver
metadata
httpManager
-- Start a background thread for listening schema sync events from other server instances,
metaVersionRef <- liftIO $ STM.newEmptyTMVarIO
-- Building the RebuildableAppContext, for more info on AppContext,
-- see note [Hasura Application State]
rebuildableAppCtxE <- liftIO $ runExceptT (buildRebuildableAppContext (logger, httpManager) serveOptions env)
!rebuildableAppCtx <- onLeft rebuildableAppCtxE $ \e -> throwErrExit InvalidEnvironmentVariableOptionsError $ T.unpack $ qeError e
-- Initialise the 'AppStateRef' from 'RebuildableSchemaCacheRef' and 'RebuildableAppContext'
appStateRef <- initialiseAppStateRef tlsAllowListRef serverMetrics rebuildableSchemaCache rebuildableAppCtx
-- An interval of 0 indicates that no schema sync is required
case soSchemaPollInterval of
Skip -> unLogger logger $ mkGenericLog @Text LevelInfo "schema-sync" "Schema sync disabled"
Interval interval -> do
unLogger logger $ mkGenericLog @String LevelInfo "schema-sync" ("Schema sync enabled. Polling at " <> show interval)
void $ startSchemaSyncListenerThread logger metadataDbPool instanceId interval metaVersionRef
subscriptionsState <- liftIO $ initSubscriptionsState logger liveQueryHook
lockedEventsCtx <- liftIO $ initLockedEventsCtx
let appEnv =
AppEnv
{ appEnvPort = soPort,
appEnvHost = soHost,
appEnvMetadataDbPool = metadataDbPool,
appEnvManager = httpManager,
appEnvLoggers = loggers,
appEnvMetadataVersionRef = metaVersionRef,
appEnvInstanceId = instanceId,
appEnvEnableMaintenanceMode = soEnableMaintenanceMode,
appEnvLoggingSettings = LoggingSettings soEnabledLogTypes soEnableMetadataQueryLogging,
appEnvEventingMode = soEventingMode,
appEnvEnableReadOnlyMode = soReadOnlyMode,
appEnvServerMetrics = serverMetrics,
appEnvShutdownLatch = latch,
appEnvMetaVersionRef = metaVersionRef,
appEnvPrometheusMetrics = prometheusMetrics,
appEnvTraceSamplingPolicy = traceSamplingPolicy,
appEnvSubscriptionState = subscriptionsState,
appEnvLockedEventsCtx = lockedEventsCtx,
appEnvConnParams = soConnParams,
appEnvTxIso = soTxIso,
appEnvConsoleAssetsDir = soConsoleAssetsDir,
appEnvConsoleSentryDsn = soConsoleSentryDsn,
appEnvConnectionOptions = soConnectionOptions,
appEnvWebSocketKeepAlive = soWebSocketKeepAlive,
appEnvWebSocketConnectionInitTimeout = soWebSocketConnectionInitTimeout,
appEnvGracefulShutdownTimeout = soGracefulShutdownTimeout,
appEnvCheckFeatureFlag = checkFeatureFlag',
appEnvSchemaPollInterval = soSchemaPollInterval
}
pure (appStateRef, appEnv)
-- | Runs catalogue migration, and returns the metadata that was fetched.
--
-- On success, this function logs the result of the migration, on failure it
-- logs a 'catalog_migrate' error and throws a fatal error.
migrateCatalogAndFetchMetadata ::
(MonadIO m, MonadBaseControl IO m) =>
Logger Hasura ->
PG.PGPool ->
Maybe (SourceConnConfiguration ('Postgres 'Vanilla)) ->
MaintenanceMode () ->
ExtensionsSchema ->
m Metadata
migrateCatalogAndFetchMetadata
logger
pool
defaultSourceConfig
maintenanceMode
extensionsSchema = do
-- TODO: should we allow the migration to happen during maintenance mode?
-- Allowing this can be a sanity check, to see if the hdb_catalog in the
-- DB has been set correctly
currentTime <- liftIO Clock.getCurrentTime
result <-
runExceptT $
PG.runTx pool (PG.Serializable, Just PG.ReadWrite) $
migrateCatalog
defaultSourceConfig
extensionsSchema
maintenanceMode
currentTime
case result of
Left err -> do
unLogger
logger
StartupLog
{ slLogLevel = LevelError,
slKind = "catalog_migrate",
slInfo = A.toJSON err
}
liftIO (throwErrJExit DatabaseMigrationError err)
Right (migrationResult, metadata) -> do
unLogger logger migrationResult
pure metadata
-- | Build the original 'RebuildableSchemaCache'.
--
-- On error, it logs a 'catalog_migrate' error and throws a fatal error. This
-- misnomer is intentional: it is to preserve a previous behaviour of the code
-- and avoid a breaking change.
buildFirstSchemaCache ::
(MonadIO m) =>
Env.Environment ->
Logger Hasura ->
ServerConfigCtx ->
SourceResolver ('Postgres 'Vanilla) ->
SourceResolver ('MSSQL) ->
Metadata ->
HTTP.Manager ->
m RebuildableSchemaCache
buildFirstSchemaCache
env
logger
serverConfigCtx
pgSourceResolver
mssqlSourceResolver
metadata
httpManager = do
let cacheBuildParams = CacheBuildParams httpManager pgSourceResolver mssqlSourceResolver serverConfigCtx
buildReason = CatalogSync
result <-
runExceptT $
runCacheBuild cacheBuildParams $
buildRebuildableSchemaCacheWithReason buildReason logger env metadata
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
-- error in the first schema cache build would be reported as
-- follows. Changing this will be a breaking change.
unLogger
logger
StartupLog
{ slLogLevel = LevelError,
slKind = "catalog_migrate",
slInfo = A.toJSON err
}
liftIO (throwErrJExit DatabaseMigrationError err)
-- | Event triggers live in the user's DB and other events
-- (cron, one-off and async actions)
-- live in the metadata DB, so we need a way to differentiate the
@ -783,13 +847,6 @@ data ShutdownAction
= EventTriggerShutdownAction (IO ())
| MetadataDBShutdownAction (ExceptT QErr IO ())
-- | If an exception is encountered , flush the log buffer and
-- rethrow If we do not flush the log buffer on exception, then log lines
-- may be missed
-- See: https://github.com/hasura/graphql-engine/issues/4772
flushLogger :: MonadIO m => LoggerCtx impl -> m ()
flushLogger = liftIO . FL.flushLogStr . _lcLoggerSet
-- | This function acts as the entrypoint for the graphql-engine webserver.
--
-- Note: at the exit of this function, or in case of a graceful server shutdown

View File

@ -4,7 +4,7 @@ module Main (main) where
import Constants qualified
import Control.Concurrent.MVar
import Control.Monad.Trans.Managed (ManagedT (..))
import Control.Monad.Trans.Managed (lowerManagedT)
import Control.Natural ((:~>) (..))
import Data.Aeson qualified as A
import Data.ByteString.Lazy.Char8 qualified as BL
@ -18,7 +18,7 @@ import Hasura.App
( AppM,
BasicConnectionInfo (..),
initMetadataConnectionInfo,
initialiseContext,
initialiseAppEnv,
mkMSSQLSourceResolver,
mkPgSourceResolver,
runAppM,
@ -121,8 +121,9 @@ main = do
ApolloFederationDisabled
cacheBuildParams = CacheBuildParams httpManager (mkPgSourceResolver print) mkMSSQLSourceResolver serverConfigCtx
(_appStateRef, appEnv) <- runManagedT
( initialiseContext
(_appInit, appEnv) <-
lowerManagedT $
initialiseAppEnv
envMap
globalCtx
serveOptions
@ -130,8 +131,6 @@ main = do
serverMetrics
prometheusMetrics
sampleAlways
)
$ \(appStateRef, appEnv) -> return (appStateRef, appEnv)
let run :: ExceptT QErr AppM a -> IO a
run =