graphql-engine/server/src-lib/Hasura/RQL/DDL/Schema/Cache/Common.hs
Auke Booij b535257251 Avoid Arrows by interpreting monads
TL;DR
---

We go from this:
```haskell
  (|
    withRecordInconsistency
      ( (|
          modifyErrA
            ( do
                (info, dependencies) <- liftEitherA -< buildRelInfo relDef
                recordDependencies -< (metadataObject, schemaObject, dependencies)
                returnA -< info
            )
        |) (addTableContext @b table . addRelationshipContext)
      )
    |) metadataObject
```
to this:
```haskell
  withRecordInconsistencyM metadataObject $ do
    modifyErr (addTableContext @b table . addRelationshipContext) $ do
      (info, dependencies) <- liftEither $ buildRelInfo relDef
      recordDependenciesM metadataObject schemaObject dependencies
      return info
```

Background
---
We use Haskell's `Arrows` language extension to gain some syntactic sugar when working with `Arrow`s. `Arrow`s are a programming abstraction comparable to `Monad`s.

Unfortunately the syntactic sugar provided by this language extension is not very sweet.

This PR shows how we can sometimes avoid using `Arrow`s altogether, without loss of functionality or correctness. It is a demo of a technique that can be used to cut down the amount of `Arrows`-based code in our codebase by about half.

Approach
---

Although _in general_ not every `Monad` is an `Arrow`, specific `Arrow` instantiations are exactly as powerful as their `Monad` equivalents. Otherwise they wouldn't be very equivalent, would they?

Just like `liftEither` interprets the `Either e` monad into an arbitrary monad implementing `MonadError e`, we add `interpA` which interprets certain concrete monads such as `Writer w` into specific arrows, e.g. ones satisfying `ArrowWriter w`. This means that the part of the code that only uses such interpretable effects can be written _monadically_, and then used in _arrow_ constructions down the line.

This approach cannot be used for arrow effects which do not have a monadic equivalent. In our codebase, the only instance of this is `ArrowCache m`, implemented by the `Rule m` arrow. So code written with `ArrowCache m` in the context cannot be rewritten monadically using this technique.

See also
---
- #1827
- #2210

PR-URL: https://github.com/hasura/graphql-engine-mono/pull/3543
Co-authored-by: jkachmar <8461423+jkachmar@users.noreply.github.com>
GitOrigin-RevId: eb79619c95f7a571bce99bc144ce42ee65d08505
2022-02-22 18:09:50 +00:00

308 lines
10 KiB
Haskell
Raw Blame History

This file contains ambiguous Unicode characters

This file contains Unicode characters that might be confused with other characters. If you think that this is intentional, you can safely ignore this warning. Use the Escape button to reveal them.

{-# LANGUAGE Arrows #-}
{-# LANGUAGE UndecidableInstances #-}
-- | Types/functions shared between modules that implement "Hasura.RQL.DDL.Schema.Cache". Other
-- modules should not import this module directly.
module Hasura.RQL.DDL.Schema.Cache.Common
( BuildOutputs (..),
CacheBuild,
CacheBuildParams (CacheBuildParams),
InvalidationKeys (..),
ikMetadata,
ikRemoteSchemas,
ikSources,
NonColumnTableInputs (..),
RebuildableSchemaCache (RebuildableSchemaCache, lastBuiltSchemaCache),
TableBuildInput (TableBuildInput, _tbiName),
TablePermissionInputs (..),
addTableContext,
bindErrorA,
boAllowlist,
boApiLimits,
boMetricsConfig,
boTlsAllowlist,
boActions,
boCronTriggers,
boCustomTypes,
boEndpoints,
boRemoteSchemas,
boRoles,
boSources,
buildInfoMap,
buildInfoMapPreservingMetadata,
initialInvalidationKeys,
invalidateKeys,
mkTableInputs,
runCacheBuild,
runCacheBuildM,
withRecordDependencies,
)
where
import Control.Arrow.Extended
import Control.Arrow.Interpret
import Control.Lens
import Control.Monad.Trans.Control (MonadBaseControl)
import Control.Monad.Unique
import Data.HashMap.Strict.Extended qualified as M
import Data.HashMap.Strict.InsOrd qualified as OMap
import Data.Sequence qualified as Seq
import Data.Text.Extended
import Hasura.Base.Error
import Hasura.Incremental qualified as Inc
import Hasura.Prelude
import Hasura.RQL.Types
import Hasura.Session
import Network.HTTP.Client.Manager (HasHttpManagerM (..))
import Network.HTTP.Client.Transformable qualified as HTTP
-- | 'InvalidationKeys' used to apply requested 'CacheInvalidations'.
data InvalidationKeys = InvalidationKeys
{ _ikMetadata :: !Inc.InvalidationKey,
_ikRemoteSchemas :: !(HashMap RemoteSchemaName Inc.InvalidationKey),
_ikSources :: !(HashMap SourceName Inc.InvalidationKey)
}
deriving (Show, Eq, Generic)
instance Inc.Cacheable InvalidationKeys
instance Inc.Select InvalidationKeys
$(makeLenses ''InvalidationKeys)
initialInvalidationKeys :: InvalidationKeys
initialInvalidationKeys = InvalidationKeys Inc.initialInvalidationKey mempty mempty
invalidateKeys :: CacheInvalidations -> InvalidationKeys -> InvalidationKeys
invalidateKeys CacheInvalidations {..} InvalidationKeys {..} =
InvalidationKeys
{ _ikMetadata = if ciMetadata then Inc.invalidate _ikMetadata else _ikMetadata,
_ikRemoteSchemas = foldl' (flip invalidate) _ikRemoteSchemas ciRemoteSchemas,
_ikSources = foldl' (flip invalidate) _ikSources ciSources
}
where
invalidate ::
(Eq a, Hashable a) =>
a ->
HashMap a Inc.InvalidationKey ->
HashMap a Inc.InvalidationKey
invalidate = M.alter $ Just . maybe Inc.initialInvalidationKey Inc.invalidate
data TableBuildInput b = TableBuildInput
{ _tbiName :: !(TableName b),
_tbiIsEnum :: !Bool,
_tbiConfiguration :: !(TableConfig b)
}
deriving (Show, Eq, Generic)
instance (Backend b) => NFData (TableBuildInput b)
instance (Backend b) => Inc.Cacheable (TableBuildInput b)
data NonColumnTableInputs b = NonColumnTableInputs
{ _nctiTable :: !(TableName b),
_nctiObjectRelationships :: ![ObjRelDef b],
_nctiArrayRelationships :: ![ArrRelDef b],
_nctiComputedFields :: ![ComputedFieldMetadata b],
_nctiRemoteRelationships :: ![RemoteRelationship]
}
deriving (Show, Eq, Generic)
data TablePermissionInputs b = TablePermissionInputs
{ _tpiTable :: !(TableName b),
_tpiInsert :: ![InsPermDef b],
_tpiSelect :: ![SelPermDef b],
_tpiUpdate :: ![UpdPermDef b],
_tpiDelete :: ![DelPermDef b]
}
deriving (Show, Eq, Generic)
instance (Backend b) => Inc.Cacheable (TablePermissionInputs b)
mkTableInputs ::
TableMetadata b -> (TableBuildInput b, NonColumnTableInputs b, TablePermissionInputs b)
mkTableInputs TableMetadata {..} =
(buildInput, nonColumns, permissions)
where
buildInput = TableBuildInput _tmTable _tmIsEnum _tmConfiguration
nonColumns =
NonColumnTableInputs
_tmTable
(OMap.elems _tmObjectRelationships)
(OMap.elems _tmArrayRelationships)
(OMap.elems _tmComputedFields)
(OMap.elems _tmRemoteRelationships)
permissions =
TablePermissionInputs
_tmTable
(OMap.elems _tmInsertPermissions)
(OMap.elems _tmSelectPermissions)
(OMap.elems _tmUpdatePermissions)
(OMap.elems _tmDeletePermissions)
-- | The direct output of 'buildSchemaCacheRule'. Contains most of the things necessary to build a
-- schema cache, but dependencies and inconsistent metadata objects are collected via a separate
-- 'MonadWriter' side channel.
data BuildOutputs = BuildOutputs
{ _boSources :: SourceCache,
_boActions :: !ActionCache,
-- | We preserve the 'MetadataObject' from the original catalog metadata in the output so we can
-- reuse it later if we need to mark the remote schema inconsistent during GraphQL schema
-- generation (because of field conflicts).
_boRemoteSchemas :: !(HashMap RemoteSchemaName (RemoteSchemaCtx, MetadataObject)),
_boAllowlist :: !InlinedAllowlist,
_boCustomTypes :: !AnnotatedCustomTypes,
_boCronTriggers :: !(M.HashMap TriggerName CronTriggerInfo),
_boEndpoints :: !(M.HashMap EndpointName (EndpointMetadata GQLQueryWithText)),
_boApiLimits :: !ApiLimit,
_boMetricsConfig :: !MetricsConfig,
_boRoles :: !(HashMap RoleName Role),
_boTlsAllowlist :: ![TlsAllow]
}
$(makeLenses ''BuildOutputs)
-- | Parameters required for schema cache build
data CacheBuildParams = CacheBuildParams
{ _cbpManager :: !HTTP.Manager,
_cbpPGSourceResolver :: !(SourceResolver ('Postgres 'Vanilla)),
_cbpMSSQLSourceResolver :: !(SourceResolver 'MSSQL),
_cbpServerConfigCtx :: !ServerConfigCtx
}
-- | The monad in which @'RebuildableSchemaCache' is being run
newtype CacheBuild a = CacheBuild (ReaderT CacheBuildParams (ExceptT QErr IO) a)
deriving
( Functor,
Applicative,
Monad,
MonadError QErr,
MonadReader CacheBuildParams,
MonadIO,
MonadBase IO,
MonadBaseControl IO,
MonadUnique
)
instance HasHttpManagerM CacheBuild where
askHttpManager = asks _cbpManager
instance HasServerConfigCtx CacheBuild where
askServerConfigCtx = asks _cbpServerConfigCtx
instance MonadResolveSource CacheBuild where
getPGSourceResolver = asks _cbpPGSourceResolver
getMSSQLSourceResolver = asks _cbpMSSQLSourceResolver
runCacheBuild ::
( MonadIO m,
MonadError QErr m
) =>
CacheBuildParams ->
CacheBuild a ->
m a
runCacheBuild params (CacheBuild m) = do
liftEitherM $ liftIO $ runExceptT (runReaderT m params)
runCacheBuildM ::
( MonadIO m,
MonadError QErr m,
HasHttpManagerM m,
HasServerConfigCtx m,
MonadResolveSource m
) =>
CacheBuild a ->
m a
runCacheBuildM m = do
params <-
CacheBuildParams
<$> askHttpManager
<*> getPGSourceResolver
<*> getMSSQLSourceResolver
<*> askServerConfigCtx
runCacheBuild params m
data RebuildableSchemaCache = RebuildableSchemaCache
{ lastBuiltSchemaCache :: !SchemaCache,
_rscInvalidationMap :: !InvalidationKeys,
_rscRebuild :: !(Inc.Rule (ReaderT BuildReason CacheBuild) (Metadata, InvalidationKeys) SchemaCache)
}
bindErrorA ::
(ArrowChoice arr, ArrowKleisli m arr, ArrowError e arr, MonadError e m) =>
arr (m a) a
bindErrorA = liftEitherA <<< arrM \m -> (Right <$> m) `catchError` (pure . Left)
{-# INLINE bindErrorA #-}
withRecordDependencies ::
(ArrowWriter (Seq CollectedInfo) arr) =>
WriterA (Seq SchemaDependency) arr (e, s) a ->
arr (e, (MetadataObject, (SchemaObjId, s))) a
withRecordDependencies f = proc (e, (metadataObject, (schemaObjectId, s))) -> do
(result, dependencies) <- runWriterA f -< (e, s)
recordDependencies -< (metadataObject, schemaObjectId, toList dependencies)
returnA -< result
{-# INLINEABLE withRecordDependencies #-}
noDuplicates ::
(MonadWriter (Seq CollectedInfo) m) =>
(a -> MetadataObject) ->
[a] ->
m (Maybe a)
noDuplicates mkMetadataObject = \case
[] -> pure Nothing
[value] -> pure $ Just value
values@(value : _) -> do
let objectId = _moId $ mkMetadataObject value
definitions = map (_moDefinition . mkMetadataObject) values
tell $ Seq.singleton $ CIInconsistency (DuplicateObjects objectId definitions)
return Nothing
-- | Processes a list of catalog metadata into a map of processed information, marking any duplicate
-- entries inconsistent.
buildInfoMap ::
( ArrowChoice arr,
Inc.ArrowDistribute arr,
ArrowWriter (Seq CollectedInfo) arr,
Eq k,
Hashable k
) =>
(a -> k) ->
(a -> MetadataObject) ->
(e, a) `arr` Maybe b ->
(e, [a]) `arr` HashMap k b
buildInfoMap extractKey mkMetadataObject buildInfo = proc (e, infos) ->
(M.groupOn extractKey infos >- returnA)
>-> (|
Inc.keyed
( \_ duplicateInfos ->
(noDuplicates mkMetadataObject duplicateInfos >- interpA @(WriterT _ Identity))
>-> (| traverseA (\info -> (e, info) >- buildInfo) |)
>-> (\info -> join info >- returnA)
)
|)
>-> (\infoMap -> M.catMaybes infoMap >- returnA)
{-# INLINEABLE buildInfoMap #-}
-- | Like 'buildInfo', but includes each processed infos associated 'MetadataObject' in the result.
-- This is useful if the results will be further processed, and the 'MetadataObject' is still needed
-- to mark the object inconsistent.
buildInfoMapPreservingMetadata ::
( ArrowChoice arr,
Inc.ArrowDistribute arr,
ArrowWriter (Seq CollectedInfo) arr,
Eq k,
Hashable k
) =>
(a -> k) ->
(a -> MetadataObject) ->
(e, a) `arr` Maybe b ->
(e, [a]) `arr` HashMap k (b, MetadataObject)
buildInfoMapPreservingMetadata extractKey mkMetadataObject buildInfo =
buildInfoMap extractKey mkMetadataObject proc (e, info) ->
((e, info) >- buildInfo) >-> \result -> result <&> (,mkMetadataObject info) >- returnA
{-# INLINEABLE buildInfoMapPreservingMetadata #-}
addTableContext :: (Backend b) => TableName b -> Text -> Text
addTableContext tableName e = "in table " <> tableName <<> ": " <> e