{-# LANGUAGE QuasiQuotes #-}
{-# LANGUAGE TemplateHaskell #-}

-- | Postgres DDL RunSQL
--
-- Escape hatch for running raw SQL against a postgres database.
--
-- 'runRunSQL' executes the provided raw SQL.
--
-- 'isSchemaCacheBuildRequiredRunSQL' checks for known schema-mutating keywords
-- in the raw SQL text.
--
-- See 'Hasura.Server.API.V2Query' and 'Hasura.Server.API.Query'.
module Hasura.Backends.Postgres.DDL.RunSQL
  ( runRunSQL,
    RunSQL (..),
    isReadOnly,
    isSchemaCacheBuildRequiredRunSQL,
  )
where

import Control.Monad.Trans.Control (MonadBaseControl)
import Data.Aeson
import Data.HashMap.Strict qualified as HashMap
import Data.HashSet qualified as HS
import Data.List.NonEmpty qualified as NE
import Data.Text.Extended
import Database.PG.Query qualified as PG
import Hasura.Backends.Postgres.Connection.MonadTx
import Hasura.Backends.Postgres.DDL.EventTrigger
import Hasura.Backends.Postgres.DDL.Source
  ( FetchFunctionMetadata,
    FetchTableMetadata,
    ToMetadataFetchQuery,
    fetchFunctionMetadata,
    fetchTableMetadata,
  )
import Hasura.Backends.Postgres.Execute.Types
import Hasura.Backends.Postgres.SQL.Types hiding (FunctionName, TableName)
import Hasura.Base.Error
import Hasura.EncJSON
import Hasura.Function.Cache
import Hasura.Prelude
import Hasura.RQL.DDL.Schema
import Hasura.RQL.DDL.Schema.Diff qualified as Diff
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.BackendType
import Hasura.RQL.Types.Column (StructuredColumnInfo (..))
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.ComputedField
import Hasura.RQL.Types.EventTrigger
import Hasura.RQL.Types.Metadata
import Hasura.RQL.Types.Metadata.Backend
import Hasura.RQL.Types.SchemaCache
import Hasura.RQL.Types.SchemaCache.Build
import Hasura.RQL.Types.SchemaCacheTypes
import Hasura.RQL.Types.Source
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.Server.Utils (quoteRegex)
import Hasura.Session
import Hasura.Table.Cache
import Hasura.Tracing qualified as Tracing
import Text.Regex.TDFA qualified as TDFA

data RunSQL = RunSQL
  { RunSQL -> Text
rSql :: Text,
    RunSQL -> SourceName
rSource :: SourceName,
    RunSQL -> Bool
rCascade :: Bool,
    RunSQL -> Maybe Bool
rCheckMetadataConsistency :: Maybe Bool,
    RunSQL -> TxAccess
rTxAccessMode :: PG.TxAccess
  }
  deriving (Int -> RunSQL -> ShowS
[RunSQL] -> ShowS
RunSQL -> String
(Int -> RunSQL -> ShowS)
-> (RunSQL -> String) -> ([RunSQL] -> ShowS) -> Show RunSQL
forall a.
(Int -> a -> ShowS) -> (a -> String) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> RunSQL -> ShowS
showsPrec :: Int -> RunSQL -> ShowS
$cshow :: RunSQL -> String
show :: RunSQL -> String
$cshowList :: [RunSQL] -> ShowS
showList :: [RunSQL] -> ShowS
Show, RunSQL -> RunSQL -> Bool
(RunSQL -> RunSQL -> Bool)
-> (RunSQL -> RunSQL -> Bool) -> Eq RunSQL
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: RunSQL -> RunSQL -> Bool
== :: RunSQL -> RunSQL -> Bool
$c/= :: RunSQL -> RunSQL -> Bool
/= :: RunSQL -> RunSQL -> Bool
Eq)

instance FromJSON RunSQL where
  parseJSON :: Value -> Parser RunSQL
parseJSON = String -> (Object -> Parser RunSQL) -> Value -> Parser RunSQL
forall a. String -> (Object -> Parser a) -> Value -> Parser a
withObject String
"RunSQL" ((Object -> Parser RunSQL) -> Value -> Parser RunSQL)
-> (Object -> Parser RunSQL) -> Value -> Parser RunSQL
forall a b. (a -> b) -> a -> b
$ \Object
o -> do
    Text
rSql <- Object
o Object -> Key -> Parser Text
forall a. FromJSON a => Object -> Key -> Parser a
.: Key
"sql"
    SourceName
rSource <- Object
o Object -> Key -> Parser (Maybe SourceName)
forall a. FromJSON a => Object -> Key -> Parser (Maybe a)
.:? Key
"source" Parser (Maybe SourceName) -> SourceName -> Parser SourceName
forall a. Parser (Maybe a) -> a -> Parser a
.!= SourceName
defaultSource
    Bool
rCascade <- Object
o Object -> Key -> Parser (Maybe Bool)
forall a. FromJSON a => Object -> Key -> Parser (Maybe a)
.:? Key
"cascade" Parser (Maybe Bool) -> Bool -> Parser Bool
forall a. Parser (Maybe a) -> a -> Parser a
.!= Bool
False
    Maybe Bool
rCheckMetadataConsistency <- Object
o Object -> Key -> Parser (Maybe Bool)
forall a. FromJSON a => Object -> Key -> Parser (Maybe a)
.:? Key
"check_metadata_consistency"
    Bool
readOnly <- Object
o Object -> Key -> Parser (Maybe Bool)
forall a. FromJSON a => Object -> Key -> Parser (Maybe a)
.:? Key
"read_only" Parser (Maybe Bool) -> Bool -> Parser Bool
forall a. Parser (Maybe a) -> a -> Parser a
.!= Bool
False
    let rTxAccessMode :: TxAccess
rTxAccessMode = if Bool
readOnly then TxAccess
PG.ReadOnly else TxAccess
PG.ReadWrite
    RunSQL -> Parser RunSQL
forall a. a -> Parser a
forall (f :: * -> *) a. Applicative f => a -> f a
pure RunSQL {Bool
Maybe Bool
Text
TxAccess
SourceName
rSql :: Text
rSource :: SourceName
rCascade :: Bool
rCheckMetadataConsistency :: Maybe Bool
rTxAccessMode :: TxAccess
rSql :: Text
rSource :: SourceName
rCascade :: Bool
rCheckMetadataConsistency :: Maybe Bool
rTxAccessMode :: TxAccess
..}

instance ToJSON RunSQL where
  toJSON :: RunSQL -> Value
toJSON RunSQL {Bool
Maybe Bool
Text
TxAccess
SourceName
rSql :: RunSQL -> Text
rSource :: RunSQL -> SourceName
rCascade :: RunSQL -> Bool
rCheckMetadataConsistency :: RunSQL -> Maybe Bool
rTxAccessMode :: RunSQL -> TxAccess
rSql :: Text
rSource :: SourceName
rCascade :: Bool
rCheckMetadataConsistency :: Maybe Bool
rTxAccessMode :: TxAccess
..} =
    [Pair] -> Value
object
      [ Key
"sql" Key -> Text -> Pair
forall kv v. (KeyValue kv, ToJSON v) => Key -> v -> kv
forall v. ToJSON v => Key -> v -> Pair
.= Text
rSql,
        Key
"source" Key -> SourceName -> Pair
forall kv v. (KeyValue kv, ToJSON v) => Key -> v -> kv
forall v. ToJSON v => Key -> v -> Pair
.= SourceName
rSource,
        Key
"cascade" Key -> Bool -> Pair
forall kv v. (KeyValue kv, ToJSON v) => Key -> v -> kv
forall v. ToJSON v => Key -> v -> Pair
.= Bool
rCascade,
        Key
"check_metadata_consistency" Key -> Maybe Bool -> Pair
forall kv v. (KeyValue kv, ToJSON v) => Key -> v -> kv
forall v. ToJSON v => Key -> v -> Pair
.= Maybe Bool
rCheckMetadataConsistency,
        Key
"read_only"
          Key -> Bool -> Pair
forall kv v. (KeyValue kv, ToJSON v) => Key -> v -> kv
forall v. ToJSON v => Key -> v -> Pair
.= case TxAccess
rTxAccessMode of
            TxAccess
PG.ReadOnly -> Bool
True
            TxAccess
PG.ReadWrite -> Bool
False
      ]

-- | Check for known schema-mutating keywords in the raw SQL text.
--
-- See Note [Checking metadata consistency in run_sql].
isSchemaCacheBuildRequiredRunSQL :: RunSQL -> Bool
isSchemaCacheBuildRequiredRunSQL :: RunSQL -> Bool
isSchemaCacheBuildRequiredRunSQL RunSQL {Bool
Maybe Bool
Text
TxAccess
SourceName
rSql :: RunSQL -> Text
rSource :: RunSQL -> SourceName
rCascade :: RunSQL -> Bool
rCheckMetadataConsistency :: RunSQL -> Maybe Bool
rTxAccessMode :: RunSQL -> TxAccess
rSql :: Text
rSource :: SourceName
rCascade :: Bool
rCheckMetadataConsistency :: Maybe Bool
rTxAccessMode :: TxAccess
..} =
  case TxAccess
rTxAccessMode of
    TxAccess
PG.ReadOnly -> Bool
False
    TxAccess
PG.ReadWrite -> Bool -> Maybe Bool -> Bool
forall a. a -> Maybe a -> a
fromMaybe (Text -> Bool
containsDDLKeyword Text
rSql) Maybe Bool
rCheckMetadataConsistency
  where
    containsDDLKeyword :: Text -> Bool
containsDDLKeyword =
      Regex -> Text -> Bool
forall regex source target.
RegexContext regex source target =>
regex -> source -> target
TDFA.match
        $$( quoteRegex
              TDFA.defaultCompOpt
                { TDFA.caseSensitive = False,
                  TDFA.multiline = True,
                  TDFA.lastStarGreedy = True
                }
              TDFA.defaultExecOpt
                { TDFA.captureGroups = False
                }
              "\\balter\\b|\\bdrop\\b|\\breplace\\b|\\bcreate function\\b|\\bcomment on\\b"
          )

isReadOnly :: RunSQL -> Bool
isReadOnly :: RunSQL -> Bool
isReadOnly RunSQL
runsql =
  case RunSQL -> TxAccess
rTxAccessMode RunSQL
runsql of
    TxAccess
PG.ReadOnly -> Bool
True
    TxAccess
PG.ReadWrite -> Bool
False

{- Note [Checking metadata consistency in run_sql]
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
SQL queries executed by run_sql may change the Postgres schema in arbitrary
ways. We attempt to automatically update the metadata to reflect those changes
as much as possible---for example, if a table is renamed, we want to update the
metadata to track the table under its new name instead of its old one. This
schema diffing (plus some integrity checking) is handled by withMetadataCheck.

But this process has overhead---it involves reloading the metadata, diffing it,
and rebuilding the schema cache---so we don’t want to do it if it isn’t
necessary. The user can explicitly disable the check via the
check_metadata_consistency option, and we also skip it if the current
transaction is in READ ONLY mode, since the schema can’t be modified in that
case, anyway.

However, even if neither read_only or check_metadata_consistency is passed, lots
of queries may not modify the schema at all. As a (fairly stupid) heuristic, we
check if the query contains any keywords for DDL operations, and if not, we skip
the metadata check as well. -}

-- | Fetch metadata of tracked tables/functions and build @'Diff.TableMeta'/@'Diff.FunctionMeta'
-- to calculate diff later in @'withMetadataCheck'.
fetchTablesFunctionsMetadata ::
  forall pgKind m.
  ( ToMetadataFetchQuery pgKind,
    FetchTableMetadata pgKind,
    FetchFunctionMetadata pgKind,
    BackendMetadata ('Postgres pgKind),
    MonadTx m
  ) =>
  TableCache ('Postgres pgKind) ->
  HS.HashSet (TableName ('Postgres pgKind)) ->
  HS.HashSet (FunctionName ('Postgres pgKind)) ->
  m ([Diff.TableMeta ('Postgres pgKind)], [Diff.FunctionMeta ('Postgres pgKind)])
fetchTablesFunctionsMetadata :: forall (pgKind :: PostgresKind) (m :: * -> *).
(ToMetadataFetchQuery pgKind, FetchTableMetadata pgKind,
 FetchFunctionMetadata pgKind, BackendMetadata ('Postgres pgKind),
 MonadTx m) =>
TableCache ('Postgres pgKind)
-> HashSet (TableName ('Postgres pgKind))
-> HashSet (FunctionName ('Postgres pgKind))
-> m ([TableMeta ('Postgres pgKind)],
      [FunctionMeta ('Postgres pgKind)])
fetchTablesFunctionsMetadata TableCache ('Postgres pgKind)
tableCache HashSet (TableName ('Postgres pgKind))
tables HashSet (FunctionName ('Postgres pgKind))
functions = do
  HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
tableMetaInfos <- HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
forall (pgKind :: PostgresKind) (m :: * -> *).
(FetchTableMetadata pgKind, Backend ('Postgres pgKind),
 ToMetadataFetchQuery pgKind, MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
forall (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
fetchTableMetadata HashSet (TableName ('Postgres pgKind))
HashSet QualifiedTable
tables
  HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionMetaInfos <- forall (pgKind :: PostgresKind) (m :: * -> *).
(FetchFunctionMetadata pgKind, MonadTx m) =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres pgKind))
fetchFunctionMetadata @pgKind HashSet (FunctionName ('Postgres pgKind))
HashSet QualifiedFunction
functions

  let functionMetas :: [FunctionMeta ('Postgres pgKind)]
functionMetas =
        [ FunctionMeta ('Postgres pgKind)
functionMeta
          | QualifiedFunction
function <- HashSet QualifiedFunction -> [QualifiedFunction]
forall a. HashSet a -> [a]
HS.toList HashSet (FunctionName ('Postgres pgKind))
HashSet QualifiedFunction
functions,
            FunctionMeta ('Postgres pgKind)
functionMeta <- HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
-> QualifiedFunction -> [FunctionMeta ('Postgres pgKind)]
mkFunctionMetas HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionMetaInfos QualifiedFunction
function
        ]
  let tableMetas :: [TableMeta ('Postgres pgKind)]
tableMetas =
        [ TableName ('Postgres pgKind)
-> DBTableMetadata ('Postgres pgKind)
-> [ComputedFieldMeta ('Postgres pgKind)]
-> TableMeta ('Postgres pgKind)
forall (b :: BackendType).
TableName b
-> DBTableMetadata b -> [ComputedFieldMeta b] -> TableMeta b
Diff.TableMeta TableName ('Postgres pgKind)
QualifiedTable
table DBTableMetadata ('Postgres pgKind)
tableMetaInfo [ComputedFieldMeta ('Postgres pgKind)]
computedFieldInfos
          | (QualifiedTable
table, DBTableMetadata ('Postgres pgKind)
tableMetaInfo) <- HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
tableMetaInfos,
            let computedFieldInfos :: [ComputedFieldMeta ('Postgres pgKind)]
computedFieldInfos =
                  [ ComputedFieldMeta ('Postgres pgKind)
computedFieldMeta
                    | Just TableInfo ('Postgres pgKind)
tableInfo <- Maybe (TableInfo ('Postgres pgKind))
-> [Maybe (TableInfo ('Postgres pgKind))]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure (QualifiedTable
-> HashMap QualifiedTable (TableInfo ('Postgres pgKind))
-> Maybe (TableInfo ('Postgres pgKind))
forall k v. (Eq k, Hashable k) => k -> HashMap k v -> Maybe v
HashMap.lookup QualifiedTable
table TableCache ('Postgres pgKind)
HashMap QualifiedTable (TableInfo ('Postgres pgKind))
tableCache),
                      ComputedFieldInfo ('Postgres pgKind)
computedField <- TableInfo ('Postgres pgKind)
-> [ComputedFieldInfo ('Postgres pgKind)]
forall (pgKind :: PostgresKind).
TableInfo ('Postgres pgKind)
-> [ComputedFieldInfo ('Postgres pgKind)]
getComputedFields TableInfo ('Postgres pgKind)
tableInfo,
                      ComputedFieldMeta ('Postgres pgKind)
computedFieldMeta <-
                        [ ComputedFieldName
-> FunctionMeta ('Postgres pgKind)
-> ComputedFieldMeta ('Postgres pgKind)
forall (b :: BackendType).
ComputedFieldName -> FunctionMeta b -> ComputedFieldMeta b
Diff.ComputedFieldMeta ComputedFieldName
fieldName FunctionMeta ('Postgres pgKind)
functionMeta
                          | let fieldName :: ComputedFieldName
fieldName = ComputedFieldInfo ('Postgres pgKind) -> ComputedFieldName
forall (b :: BackendType). ComputedFieldInfo b -> ComputedFieldName
_cfiName ComputedFieldInfo ('Postgres pgKind)
computedField
                                function :: FunctionName ('Postgres pgKind)
function = ComputedFieldFunction ('Postgres pgKind)
-> FunctionName ('Postgres pgKind)
forall (b :: BackendType).
ComputedFieldFunction b -> FunctionName b
_cffName (ComputedFieldFunction ('Postgres pgKind)
 -> FunctionName ('Postgres pgKind))
-> ComputedFieldFunction ('Postgres pgKind)
-> FunctionName ('Postgres pgKind)
forall a b. (a -> b) -> a -> b
$ ComputedFieldInfo ('Postgres pgKind)
-> ComputedFieldFunction ('Postgres pgKind)
forall (b :: BackendType).
ComputedFieldInfo b -> ComputedFieldFunction b
_cfiFunction ComputedFieldInfo ('Postgres pgKind)
computedField,
                            FunctionMeta ('Postgres pgKind)
functionMeta <- HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
-> QualifiedFunction -> [FunctionMeta ('Postgres pgKind)]
mkFunctionMetas HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionMetaInfos FunctionName ('Postgres pgKind)
QualifiedFunction
function
                        ]
                  ]
        ]

  ([TableMeta ('Postgres pgKind)], [FunctionMeta ('Postgres pgKind)])
-> m ([TableMeta ('Postgres pgKind)],
      [FunctionMeta ('Postgres pgKind)])
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ([TableMeta ('Postgres pgKind)]
tableMetas, [FunctionMeta ('Postgres pgKind)]
functionMetas)
  where
    mkFunctionMetas ::
      HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind)) ->
      QualifiedFunction ->
      [Diff.FunctionMeta ('Postgres pgKind)]
    mkFunctionMetas :: HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
-> QualifiedFunction -> [FunctionMeta ('Postgres pgKind)]
mkFunctionMetas HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionMetaInfos QualifiedFunction
function =
      [ OID
-> FunctionName ('Postgres pgKind)
-> FunctionVolatility
-> FunctionMeta ('Postgres pgKind)
forall (b :: BackendType).
OID -> FunctionName b -> FunctionVolatility -> FunctionMeta b
Diff.FunctionMeta (PGRawFunctionInfo -> OID
rfiOid PGRawFunctionInfo
rawInfo) FunctionName ('Postgres pgKind)
QualifiedFunction
function (PGRawFunctionInfo -> FunctionVolatility
rfiFunctionType PGRawFunctionInfo
rawInfo)
        | -- It would seem like we could feasibly detect function overloads here already,
          -- But that is handled elsewhere.
          Just FunctionOverloads ('Postgres pgKind)
overloads <- Maybe (FunctionOverloads ('Postgres pgKind))
-> [Maybe (FunctionOverloads ('Postgres pgKind))]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure (QualifiedFunction
-> HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
-> Maybe (FunctionOverloads ('Postgres pgKind))
forall k v. (Eq k, Hashable k) => k -> HashMap k v -> Maybe v
HashMap.lookup QualifiedFunction
function HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionMetaInfos),
          PGRawFunctionInfo
rawInfo <- NonEmpty PGRawFunctionInfo -> [PGRawFunctionInfo]
forall a. NonEmpty a -> [a]
NE.toList (NonEmpty PGRawFunctionInfo -> [PGRawFunctionInfo])
-> NonEmpty PGRawFunctionInfo -> [PGRawFunctionInfo]
forall a b. (a -> b) -> a -> b
$ FunctionOverloads ('Postgres pgKind)
-> NonEmpty (RawFunctionInfo ('Postgres pgKind))
forall (b :: BackendType).
FunctionOverloads b -> NonEmpty (RawFunctionInfo b)
getFunctionOverloads FunctionOverloads ('Postgres pgKind)
overloads
      ]

-- | Used as an escape hatch to run raw SQL against a database.
runRunSQL ::
  forall (pgKind :: PostgresKind) m.
  ( BackendMetadata ('Postgres pgKind),
    ToMetadataFetchQuery pgKind,
    FetchTableMetadata pgKind,
    FetchFunctionMetadata pgKind,
    CacheRWM m,
    MetadataM m,
    MonadBaseControl IO m,
    MonadError QErr m,
    MonadIO m,
    Tracing.MonadTrace m,
    UserInfoM m
  ) =>
  SQLGenCtx ->
  RunSQL ->
  m EncJSON
runRunSQL :: forall (pgKind :: PostgresKind) (m :: * -> *).
(BackendMetadata ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 FetchTableMetadata pgKind, FetchFunctionMetadata pgKind,
 CacheRWM m, MetadataM m, MonadBaseControl IO m, MonadError QErr m,
 MonadIO m, MonadTrace m, UserInfoM m) =>
SQLGenCtx -> RunSQL -> m EncJSON
runRunSQL SQLGenCtx
sqlGen q :: RunSQL
q@RunSQL {Bool
Maybe Bool
Text
TxAccess
SourceName
rSql :: RunSQL -> Text
rSource :: RunSQL -> SourceName
rCascade :: RunSQL -> Bool
rCheckMetadataConsistency :: RunSQL -> Maybe Bool
rTxAccessMode :: RunSQL -> TxAccess
rSql :: Text
rSource :: SourceName
rCascade :: Bool
rCheckMetadataConsistency :: Maybe Bool
rTxAccessMode :: TxAccess
..} = do
  PGSourceConfig
sourceConfig <- forall (b :: BackendType) (m :: * -> *).
(CacheRM m, MonadError QErr m, Backend b, MetadataM m) =>
SourceName -> m (SourceConfig b)
askSourceConfig @('Postgres pgKind) SourceName
rSource
  Maybe TraceContext
traceCtx <- m (Maybe TraceContext)
forall (m :: * -> *). MonadTrace m => m (Maybe TraceContext)
Tracing.currentContext
  UserInfo
userInfo <- m UserInfo
forall (m :: * -> *). UserInfoM m => m UserInfo
askUserInfo
  let pgExecCtx :: PGExecCtx
pgExecCtx = PGSourceConfig -> PGExecCtx
_pscExecCtx PGSourceConfig
sourceConfig
  if (RunSQL -> Bool
isSchemaCacheBuildRequiredRunSQL RunSQL
q)
    then do
      -- see Note [Checking metadata consistency in run_sql]
      forall (pgKind :: PostgresKind) a (m :: * -> *).
(BackendMetadata ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 FetchTableMetadata pgKind, FetchFunctionMetadata pgKind,
 CacheRWM m, MetadataM m, MonadBaseControl IO m, MonadError QErr m,
 MonadIO m) =>
SQLGenCtx -> SourceName -> Bool -> TxAccess -> TxET QErr m a -> m a
withMetadataCheck @pgKind SQLGenCtx
sqlGen SourceName
rSource Bool
rCascade TxAccess
rTxAccessMode
        (TxET QErr m EncJSON -> m EncJSON)
-> TxET QErr m EncJSON -> m EncJSON
forall a b. (a -> b) -> a -> b
$ Maybe TraceContext -> TxET QErr m EncJSON -> TxET QErr m EncJSON
forall (m :: * -> *) a.
MonadIO m =>
Maybe TraceContext -> TxET QErr m a -> TxET QErr m a
withTraceContext Maybe TraceContext
traceCtx
        (TxET QErr m EncJSON -> TxET QErr m EncJSON)
-> TxET QErr m EncJSON -> TxET QErr m EncJSON
forall a b. (a -> b) -> a -> b
$ UserInfo -> TxET QErr m EncJSON -> TxET QErr m EncJSON
forall (m :: * -> *) a.
MonadIO m =>
UserInfo -> TxET QErr m a -> TxET QErr m a
withUserInfo UserInfo
userInfo
        (TxET QErr m EncJSON -> TxET QErr m EncJSON)
-> TxET QErr m EncJSON -> TxET QErr m EncJSON
forall a b. (a -> b) -> a -> b
$ Text -> TxET QErr m EncJSON
forall (n :: * -> *). MonadTx n => Text -> n EncJSON
execRawSQL Text
rSql
    else do
      PGExecCtx
-> PGExecTxType -> PGExecFrom -> TxET QErr m EncJSON -> m EncJSON
forall (m :: * -> *) a.
(MonadIO m, MonadBaseControl IO m, MonadError QErr m, MonadTrace m,
 UserInfoM m) =>
PGExecCtx -> PGExecTxType -> PGExecFrom -> TxET QErr m a -> m a
runTxWithCtx PGExecCtx
pgExecCtx (TxAccess -> Maybe TxIsolation -> PGExecTxType
Tx TxAccess
rTxAccessMode Maybe TxIsolation
forall a. Maybe a
Nothing) PGExecFrom
RunSQLQuery (TxET QErr m EncJSON -> m EncJSON)
-> TxET QErr m EncJSON -> m EncJSON
forall a b. (a -> b) -> a -> b
$ Text -> TxET QErr m EncJSON
forall (n :: * -> *). MonadTx n => Text -> n EncJSON
execRawSQL Text
rSql
  where
    execRawSQL :: (MonadTx n) => Text -> n EncJSON
    execRawSQL :: forall (n :: * -> *). MonadTx n => Text -> n EncJSON
execRawSQL =
      (RunSQLRes -> EncJSON) -> n RunSQLRes -> n EncJSON
forall a b. (a -> b) -> n a -> n b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (forall a. ToJSON a => a -> EncJSON
encJFromJValue @RunSQLRes) (n RunSQLRes -> n EncJSON)
-> (Text -> n RunSQLRes) -> Text -> n EncJSON
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TxE QErr RunSQLRes -> n RunSQLRes
forall a. TxE QErr a -> n a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx (TxE QErr RunSQLRes -> n RunSQLRes)
-> (Text -> TxE QErr RunSQLRes) -> Text -> n RunSQLRes
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (PGTxErr -> QErr) -> Query -> TxE QErr RunSQLRes
forall (m :: * -> *) a e.
(MonadIO m, FromRes a) =>
(PGTxErr -> e) -> Query -> TxET e m a
PG.multiQE PGTxErr -> QErr
forall {a}. ToJSON a => a -> QErr
rawSqlErrHandler (Query -> TxE QErr RunSQLRes)
-> (Text -> Query) -> Text -> TxE QErr RunSQLRes
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Text -> Query
PG.fromText
      where
        rawSqlErrHandler :: a -> QErr
rawSqlErrHandler a
txe =
          (Code -> Text -> QErr
err400 Code
PostgresError Text
"query execution failed") {qeInternal :: Maybe QErrExtra
qeInternal = QErrExtra -> Maybe QErrExtra
forall a. a -> Maybe a
Just (QErrExtra -> Maybe QErrExtra) -> QErrExtra -> Maybe QErrExtra
forall a b. (a -> b) -> a -> b
$ Value -> QErrExtra
ExtraInternal (Value -> QErrExtra) -> Value -> QErrExtra
forall a b. (a -> b) -> a -> b
$ a -> Value
forall a. ToJSON a => a -> Value
toJSON a
txe}

-- | @'withMetadataCheck' source cascade txAccess runSQLQuery@ executes @runSQLQuery@ and checks if the schema changed as a
-- result. If it did, it checks to ensure the changes do not violate any integrity constraints, and
-- if not, incorporates them into the schema cache.
-- TODO(antoine): shouldn't this be generalized?
withMetadataCheck ::
  forall (pgKind :: PostgresKind) a m.
  ( BackendMetadata ('Postgres pgKind),
    ToMetadataFetchQuery pgKind,
    FetchTableMetadata pgKind,
    FetchFunctionMetadata pgKind,
    CacheRWM m,
    MetadataM m,
    MonadBaseControl IO m,
    MonadError QErr m,
    MonadIO m
  ) =>
  SQLGenCtx ->
  SourceName ->
  Bool ->
  PG.TxAccess ->
  PG.TxET QErr m a ->
  m a
withMetadataCheck :: forall (pgKind :: PostgresKind) a (m :: * -> *).
(BackendMetadata ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 FetchTableMetadata pgKind, FetchFunctionMetadata pgKind,
 CacheRWM m, MetadataM m, MonadBaseControl IO m, MonadError QErr m,
 MonadIO m) =>
SQLGenCtx -> SourceName -> Bool -> TxAccess -> TxET QErr m a -> m a
withMetadataCheck SQLGenCtx
sqlGen SourceName
source Bool
cascade TxAccess
txAccess TxET QErr m a
runSQLQuery = do
  SourceInfo {Maybe QueryTagsConfig
TableCache ('Postgres pgKind)
FunctionCache ('Postgres pgKind)
StoredProcedureCache ('Postgres pgKind)
LogicalModelCache ('Postgres pgKind)
NativeQueryCache ('Postgres pgKind)
BackendSourceKind ('Postgres pgKind)
SourceName
SourceConfig ('Postgres pgKind)
ResolvedSourceCustomization
DBObjectsIntrospection ('Postgres pgKind)
_siName :: SourceName
_siSourceKind :: BackendSourceKind ('Postgres pgKind)
_siTables :: TableCache ('Postgres pgKind)
_siFunctions :: FunctionCache ('Postgres pgKind)
_siNativeQueries :: NativeQueryCache ('Postgres pgKind)
_siStoredProcedures :: StoredProcedureCache ('Postgres pgKind)
_siLogicalModels :: LogicalModelCache ('Postgres pgKind)
_siConfiguration :: SourceConfig ('Postgres pgKind)
_siQueryTagsConfig :: Maybe QueryTagsConfig
_siCustomization :: ResolvedSourceCustomization
_siDbObjectsIntrospection :: DBObjectsIntrospection ('Postgres pgKind)
_siName :: forall (b :: BackendType). SourceInfo b -> SourceName
_siSourceKind :: forall (b :: BackendType). SourceInfo b -> BackendSourceKind b
_siTables :: forall (b :: BackendType). SourceInfo b -> TableCache b
_siFunctions :: forall (b :: BackendType). SourceInfo b -> FunctionCache b
_siNativeQueries :: forall (b :: BackendType). SourceInfo b -> NativeQueryCache b
_siStoredProcedures :: forall (b :: BackendType). SourceInfo b -> StoredProcedureCache b
_siLogicalModels :: forall (b :: BackendType). SourceInfo b -> LogicalModelCache b
_siConfiguration :: forall (b :: BackendType). SourceInfo b -> SourceConfig b
_siQueryTagsConfig :: forall (b :: BackendType). SourceInfo b -> Maybe QueryTagsConfig
_siCustomization :: forall (b :: BackendType).
SourceInfo b -> ResolvedSourceCustomization
_siDbObjectsIntrospection :: forall (b :: BackendType). SourceInfo b -> DBObjectsIntrospection b
..} <- forall (b :: BackendType) (m :: * -> *).
(CacheRM m, MetadataM m, MonadError QErr m, Backend b) =>
SourceName -> m (SourceInfo b)
askSourceInfo @('Postgres pgKind) SourceName
source

  -- Run SQL query and metadata checker in a transaction
  (a
queryResult, MetadataModifier
metadataUpdater) <- SourceName
-> SourceConfig ('Postgres pgKind)
-> TxAccess
-> TableCache ('Postgres pgKind)
-> FunctionCache ('Postgres pgKind)
-> Bool
-> TxET QErr m a
-> m (a, MetadataModifier)
forall (m :: * -> *) a (pgKind :: PostgresKind).
(BackendMetadata ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 FetchTableMetadata pgKind, FetchFunctionMetadata pgKind,
 CacheRWM m, MonadIO m, MonadBaseControl IO m, MonadError QErr m) =>
SourceName
-> SourceConfig ('Postgres pgKind)
-> TxAccess
-> TableCache ('Postgres pgKind)
-> FunctionCache ('Postgres pgKind)
-> Bool
-> TxET QErr m a
-> m (a, MetadataModifier)
runTxWithMetadataCheck SourceName
source SourceConfig ('Postgres pgKind)
_siConfiguration TxAccess
txAccess TableCache ('Postgres pgKind)
_siTables FunctionCache ('Postgres pgKind)
_siFunctions Bool
cascade TxET QErr m a
runSQLQuery

  -- Build schema cache with updated metadata
  m () -> m ()
forall (m :: * -> *) a. (QErrM m, CacheRM m) => m a -> m a
withNewInconsistentObjsCheck
    (m () -> m ()) -> m () -> m ()
forall a b. (a -> b) -> a -> b
$ CacheInvalidations -> MetadataModifier -> m ()
forall (m :: * -> *).
(MetadataM m, CacheRWM m) =>
CacheInvalidations -> MetadataModifier -> m ()
buildSchemaCacheWithInvalidations CacheInvalidations
forall a. Monoid a => a
mempty {ciSources :: HashSet SourceName
ciSources = SourceName -> HashSet SourceName
forall a. Hashable a => a -> HashSet a
HS.singleton SourceName
source} MetadataModifier
metadataUpdater

  SchemaCache
postRunSQLSchemaCache <- m SchemaCache
forall (m :: * -> *). CacheRM m => m SchemaCache
askSchemaCache

  -- Recreate event triggers in hdb_catalog. Event triggers are dropped before executing @'runSQLQuery'.
  PGSourceConfig -> SchemaCache -> m ()
recreateEventTriggers SourceConfig ('Postgres pgKind)
PGSourceConfig
_siConfiguration SchemaCache
postRunSQLSchemaCache

  a -> m a
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure a
queryResult
  where
    recreateEventTriggers :: PGSourceConfig -> SchemaCache -> m ()
    recreateEventTriggers :: PGSourceConfig -> SchemaCache -> m ()
recreateEventTriggers PGSourceConfig
sourceConfig SchemaCache
schemaCache = do
      let tables :: HashMap QualifiedTable (TableInfo ('Postgres pgKind))
tables = HashMap QualifiedTable (TableInfo ('Postgres pgKind))
-> Maybe (HashMap QualifiedTable (TableInfo ('Postgres pgKind)))
-> HashMap QualifiedTable (TableInfo ('Postgres pgKind))
forall a. a -> Maybe a -> a
fromMaybe HashMap QualifiedTable (TableInfo ('Postgres pgKind))
forall a. Monoid a => a
mempty (Maybe (HashMap QualifiedTable (TableInfo ('Postgres pgKind)))
 -> HashMap QualifiedTable (TableInfo ('Postgres pgKind)))
-> Maybe (HashMap QualifiedTable (TableInfo ('Postgres pgKind)))
-> HashMap QualifiedTable (TableInfo ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ forall (b :: BackendType).
Backend b =>
SourceName -> SourceCache -> Maybe (TableCache b)
unsafeTableCache @('Postgres pgKind) SourceName
source (SourceCache -> Maybe (TableCache ('Postgres pgKind)))
-> SourceCache -> Maybe (TableCache ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ SchemaCache -> SourceCache
scSources SchemaCache
schemaCache
      m (Either QErr ()) -> m ()
forall e (m :: * -> *) a. MonadError e m => m (Either e a) -> m a
liftEitherM
        (m (Either QErr ()) -> m ()) -> m (Either QErr ()) -> m ()
forall a b. (a -> b) -> a -> b
$ PGSourceConfig
-> PGExecFrom -> TxET QErr m () -> m (Either QErr ())
forall (m :: * -> *) a.
(MonadIO m, MonadBaseControl IO m) =>
PGSourceConfig -> PGExecFrom -> TxET QErr m a -> m (Either QErr a)
runPgSourceWriteTx PGSourceConfig
sourceConfig PGExecFrom
RunSQLQuery
        (TxET QErr m () -> m (Either QErr ()))
-> TxET QErr m () -> m (Either QErr ())
forall a b. (a -> b) -> a -> b
$ [TableInfo ('Postgres pgKind)]
-> (TableInfo ('Postgres pgKind) -> TxET QErr m ())
-> TxET QErr m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
t a -> (a -> m b) -> m ()
forM_ (HashMap QualifiedTable (TableInfo ('Postgres pgKind))
-> [TableInfo ('Postgres pgKind)]
forall k v. HashMap k v -> [v]
HashMap.elems HashMap QualifiedTable (TableInfo ('Postgres pgKind))
tables)
        ((TableInfo ('Postgres pgKind) -> TxET QErr m ())
 -> TxET QErr m ())
-> (TableInfo ('Postgres pgKind) -> TxET QErr m ())
-> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ \(TableInfo TableCoreInfo ('Postgres pgKind)
coreInfo RolePermInfoMap ('Postgres pgKind)
_ EventTriggerInfoMap ('Postgres pgKind)
eventTriggers RolePermInfo ('Postgres pgKind)
_) -> do
          let table :: TableName ('Postgres pgKind)
table = TableCoreInfo ('Postgres pgKind) -> TableName ('Postgres pgKind)
forall (b :: BackendType) field primaryKeyColumn.
TableCoreInfoG b field primaryKeyColumn -> TableName b
_tciName TableCoreInfo ('Postgres pgKind)
coreInfo
              columns :: [ColumnInfo ('Postgres pgKind)]
columns = (StructuredColumnInfo ('Postgres pgKind)
 -> ColumnInfo ('Postgres pgKind))
-> [StructuredColumnInfo ('Postgres pgKind)]
-> [ColumnInfo ('Postgres pgKind)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(SCIScalarColumn ColumnInfo ('Postgres pgKind)
col) -> ColumnInfo ('Postgres pgKind)
col) ([StructuredColumnInfo ('Postgres pgKind)]
 -> [ColumnInfo ('Postgres pgKind)])
-> [StructuredColumnInfo ('Postgres pgKind)]
-> [ColumnInfo ('Postgres pgKind)]
forall a b. (a -> b) -> a -> b
$ FieldInfoMap (FieldInfo ('Postgres pgKind))
-> [StructuredColumnInfo ('Postgres pgKind)]
forall (backend :: BackendType).
FieldInfoMap (FieldInfo backend) -> [StructuredColumnInfo backend]
getCols (FieldInfoMap (FieldInfo ('Postgres pgKind))
 -> [StructuredColumnInfo ('Postgres pgKind)])
-> FieldInfoMap (FieldInfo ('Postgres pgKind))
-> [StructuredColumnInfo ('Postgres pgKind)]
forall a b. (a -> b) -> a -> b
$ TableCoreInfo ('Postgres pgKind)
-> FieldInfoMap (FieldInfo ('Postgres pgKind))
forall (b :: BackendType) field primaryKeyColumn.
TableCoreInfoG b field primaryKeyColumn -> FieldInfoMap field
_tciFieldInfoMap TableCoreInfo ('Postgres pgKind)
coreInfo
          [(TriggerName, EventTriggerInfo ('Postgres pgKind))]
-> ((TriggerName, EventTriggerInfo ('Postgres pgKind))
    -> TxET QErr m ())
-> TxET QErr m ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
t a -> (a -> m b) -> m ()
forM_ (EventTriggerInfoMap ('Postgres pgKind)
-> [(TriggerName, EventTriggerInfo ('Postgres pgKind))]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList EventTriggerInfoMap ('Postgres pgKind)
eventTriggers) (((TriggerName, EventTriggerInfo ('Postgres pgKind))
  -> TxET QErr m ())
 -> TxET QErr m ())
-> ((TriggerName, EventTriggerInfo ('Postgres pgKind))
    -> TxET QErr m ())
-> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ \(TriggerName
triggerName, EventTriggerInfo {TriggerOpsDef ('Postgres pgKind)
etiOpsDef :: TriggerOpsDef ('Postgres pgKind)
etiOpsDef :: forall (b :: BackendType). EventTriggerInfo b -> TriggerOpsDef b
etiOpsDef, TriggerOnReplication
etiTriggerOnReplication :: TriggerOnReplication
etiTriggerOnReplication :: forall (b :: BackendType).
EventTriggerInfo b -> TriggerOnReplication
etiTriggerOnReplication}) -> do
            (ReaderT SQLGenCtx (TxET QErr m) () -> SQLGenCtx -> TxET QErr m ())
-> SQLGenCtx
-> ReaderT SQLGenCtx (TxET QErr m) ()
-> TxET QErr m ()
forall a b c. (a -> b -> c) -> b -> a -> c
flip ReaderT SQLGenCtx (TxET QErr m) () -> SQLGenCtx -> TxET QErr m ()
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT SQLGenCtx
sqlGen
              (ReaderT SQLGenCtx (TxET QErr m) () -> TxET QErr m ())
-> ReaderT SQLGenCtx (TxET QErr m) () -> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ TriggerName
-> QualifiedTable
-> TriggerOnReplication
-> [ColumnInfo ('Postgres pgKind)]
-> TriggerOpsDef ('Postgres pgKind)
-> ReaderT SQLGenCtx (TxET QErr m) ()
forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), MonadTx m, MonadReader SQLGenCtx m) =>
TriggerName
-> QualifiedTable
-> TriggerOnReplication
-> [ColumnInfo ('Postgres pgKind)]
-> TriggerOpsDef ('Postgres pgKind)
-> m ()
mkAllTriggersQ TriggerName
triggerName TableName ('Postgres pgKind)
QualifiedTable
table TriggerOnReplication
etiTriggerOnReplication [ColumnInfo ('Postgres pgKind)]
columns TriggerOpsDef ('Postgres pgKind)
etiOpsDef

-- | @'runTxWithMetadataCheck source sourceConfig txAccess tableCache functionCache cascadeDependencies tx' checks for
-- changes in GraphQL Engine metadata when a @'tx' is executed on the database alters Postgres
-- schema of tables and functions. If any indirect dependencies (Eg. remote table dependence of a relationship) are
-- found and @'cascadeDependencies' is False, then an exception is raised.
runTxWithMetadataCheck ::
  forall m a (pgKind :: PostgresKind).
  ( BackendMetadata ('Postgres pgKind),
    ToMetadataFetchQuery pgKind,
    FetchTableMetadata pgKind,
    FetchFunctionMetadata pgKind,
    CacheRWM m,
    MonadIO m,
    MonadBaseControl IO m,
    MonadError QErr m
  ) =>
  SourceName ->
  SourceConfig ('Postgres pgKind) ->
  PG.TxAccess ->
  TableCache ('Postgres pgKind) ->
  FunctionCache ('Postgres pgKind) ->
  Bool ->
  PG.TxET QErr m a ->
  m (a, MetadataModifier)
runTxWithMetadataCheck :: forall (m :: * -> *) a (pgKind :: PostgresKind).
(BackendMetadata ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 FetchTableMetadata pgKind, FetchFunctionMetadata pgKind,
 CacheRWM m, MonadIO m, MonadBaseControl IO m, MonadError QErr m) =>
SourceName
-> SourceConfig ('Postgres pgKind)
-> TxAccess
-> TableCache ('Postgres pgKind)
-> FunctionCache ('Postgres pgKind)
-> Bool
-> TxET QErr m a
-> m (a, MetadataModifier)
runTxWithMetadataCheck SourceName
source SourceConfig ('Postgres pgKind)
sourceConfig TxAccess
txAccess TableCache ('Postgres pgKind)
tableCache FunctionCache ('Postgres pgKind)
functionCache Bool
cascadeDependencies TxET QErr m a
tx =
  m (Either QErr (a, MetadataModifier)) -> m (a, MetadataModifier)
forall e (m :: * -> *) a. MonadError e m => m (Either e a) -> m a
liftEitherM
    (m (Either QErr (a, MetadataModifier)) -> m (a, MetadataModifier))
-> m (Either QErr (a, MetadataModifier)) -> m (a, MetadataModifier)
forall a b. (a -> b) -> a -> b
$ ExceptT QErr m (a, MetadataModifier)
-> m (Either QErr (a, MetadataModifier))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT
    (ExceptT QErr m (a, MetadataModifier)
 -> m (Either QErr (a, MetadataModifier)))
-> ExceptT QErr m (a, MetadataModifier)
-> m (Either QErr (a, MetadataModifier))
forall a b. (a -> b) -> a -> b
$ PGExecCtx -> PGExecCtxInfo -> RunTx
_pecRunTx (PGSourceConfig -> PGExecCtx
_pscExecCtx SourceConfig ('Postgres pgKind)
PGSourceConfig
sourceConfig) (PGExecTxType -> PGExecFrom -> PGExecCtxInfo
PGExecCtxInfo (TxAccess -> Maybe TxIsolation -> PGExecTxType
Tx TxAccess
txAccess Maybe TxIsolation
forall a. Maybe a
Nothing) PGExecFrom
RunSQLQuery)
    (TxET QErr m (a, MetadataModifier)
 -> ExceptT QErr m (a, MetadataModifier))
-> TxET QErr m (a, MetadataModifier)
-> ExceptT QErr m (a, MetadataModifier)
forall a b. (a -> b) -> a -> b
$ do
      -- Running in a transaction helps to rollback the @'tx' execution in case of any exceptions

      -- Before running the @'tx', fetch metadata of existing tables and functions from Postgres.
      let tableNames :: HashSet QualifiedTable
tableNames = HashMap QualifiedTable (TableInfo ('Postgres pgKind))
-> HashSet QualifiedTable
forall k a. HashMap k a -> HashSet k
HashMap.keysSet TableCache ('Postgres pgKind)
HashMap QualifiedTable (TableInfo ('Postgres pgKind))
tableCache
          computedFieldFunctions :: HashSet QualifiedFunction
computedFieldFunctions = [HashSet QualifiedFunction] -> HashSet QualifiedFunction
forall a. Monoid a => [a] -> a
mconcat ([HashSet QualifiedFunction] -> HashSet QualifiedFunction)
-> [HashSet QualifiedFunction] -> HashSet QualifiedFunction
forall a b. (a -> b) -> a -> b
$ (TableInfo ('Postgres pgKind) -> HashSet QualifiedFunction)
-> [TableInfo ('Postgres pgKind)] -> [HashSet QualifiedFunction]
forall a b. (a -> b) -> [a] -> [b]
map TableInfo ('Postgres pgKind)
-> HashSet (FunctionName ('Postgres pgKind))
TableInfo ('Postgres pgKind) -> HashSet QualifiedFunction
forall (pgKind :: PostgresKind).
TableInfo ('Postgres pgKind)
-> HashSet (FunctionName ('Postgres pgKind))
getComputedFieldFunctions (HashMap QualifiedTable (TableInfo ('Postgres pgKind))
-> [TableInfo ('Postgres pgKind)]
forall k v. HashMap k v -> [v]
HashMap.elems TableCache ('Postgres pgKind)
HashMap QualifiedTable (TableInfo ('Postgres pgKind))
tableCache)
          functionNames :: HashSet QualifiedFunction
functionNames = HashMap QualifiedFunction (FunctionInfo ('Postgres pgKind))
-> HashSet QualifiedFunction
forall k a. HashMap k a -> HashSet k
HashMap.keysSet FunctionCache ('Postgres pgKind)
HashMap QualifiedFunction (FunctionInfo ('Postgres pgKind))
functionCache HashSet QualifiedFunction
-> HashSet QualifiedFunction -> HashSet QualifiedFunction
forall a. Semigroup a => a -> a -> a
<> HashSet QualifiedFunction
computedFieldFunctions
      ([TableMeta ('Postgres pgKind)]
preTxTablesMeta, [FunctionMeta ('Postgres pgKind)]
preTxFunctionsMeta) <- TableCache ('Postgres pgKind)
-> HashSet (TableName ('Postgres pgKind))
-> HashSet (FunctionName ('Postgres pgKind))
-> TxET
     QErr
     m
     ([TableMeta ('Postgres pgKind)], [FunctionMeta ('Postgres pgKind)])
forall (pgKind :: PostgresKind) (m :: * -> *).
(ToMetadataFetchQuery pgKind, FetchTableMetadata pgKind,
 FetchFunctionMetadata pgKind, BackendMetadata ('Postgres pgKind),
 MonadTx m) =>
TableCache ('Postgres pgKind)
-> HashSet (TableName ('Postgres pgKind))
-> HashSet (FunctionName ('Postgres pgKind))
-> m ([TableMeta ('Postgres pgKind)],
      [FunctionMeta ('Postgres pgKind)])
fetchTablesFunctionsMetadata TableCache ('Postgres pgKind)
tableCache HashSet (TableName ('Postgres pgKind))
HashSet QualifiedTable
tableNames HashSet (FunctionName ('Postgres pgKind))
HashSet QualifiedFunction
functionNames

      -- Since the @'tx' may alter table/function names we use the OIDs of underlying tables
      -- (sourced from 'pg_class' for tables and 'pg_proc' for functions), which remain unchanged in the
      -- case if a table/function is renamed.
      let tableOids :: HashSet OID
tableOids = [OID] -> HashSet OID
forall a. (Eq a, Hashable a) => [a] -> HashSet a
HS.fromList ([OID] -> HashSet OID) -> [OID] -> HashSet OID
forall a b. (a -> b) -> a -> b
$ (TableMeta ('Postgres pgKind) -> OID)
-> [TableMeta ('Postgres pgKind)] -> [OID]
forall a b. (a -> b) -> [a] -> [b]
map (DBTableMetadata ('Postgres pgKind) -> OID
forall (b :: BackendType). DBTableMetadata b -> OID
_ptmiOid (DBTableMetadata ('Postgres pgKind) -> OID)
-> (TableMeta ('Postgres pgKind)
    -> DBTableMetadata ('Postgres pgKind))
-> TableMeta ('Postgres pgKind)
-> OID
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TableMeta ('Postgres pgKind) -> DBTableMetadata ('Postgres pgKind)
forall (b :: BackendType). TableMeta b -> DBTableMetadata b
Diff.tmInfo) [TableMeta ('Postgres pgKind)]
preTxTablesMeta
          functionOids :: HashSet OID
functionOids = [OID] -> HashSet OID
forall a. (Eq a, Hashable a) => [a] -> HashSet a
HS.fromList ([OID] -> HashSet OID) -> [OID] -> HashSet OID
forall a b. (a -> b) -> a -> b
$ (FunctionMeta ('Postgres pgKind) -> OID)
-> [FunctionMeta ('Postgres pgKind)] -> [OID]
forall a b. (a -> b) -> [a] -> [b]
map FunctionMeta ('Postgres pgKind) -> OID
forall (b :: BackendType). FunctionMeta b -> OID
Diff.fmOid [FunctionMeta ('Postgres pgKind)]
preTxFunctionsMeta

      -- Run the transaction
      a
txResult <- TxET QErr m a
tx

      ([TableMeta ('Postgres pgKind)]
postTxTablesMeta, [FunctionMeta ('Postgres pgKind)]
postTxFunctionMeta) <-
        (HashSet QualifiedTable
 -> HashSet QualifiedFunction
 -> TxET
      QErr
      m
      ([TableMeta ('Postgres pgKind)],
       [FunctionMeta ('Postgres pgKind)]))
-> (HashSet QualifiedTable, HashSet QualifiedFunction)
-> TxET
     QErr
     m
     ([TableMeta ('Postgres pgKind)], [FunctionMeta ('Postgres pgKind)])
forall a b c. (a -> b -> c) -> (a, b) -> c
uncurry (TableCache ('Postgres pgKind)
-> HashSet (TableName ('Postgres pgKind))
-> HashSet (FunctionName ('Postgres pgKind))
-> TxET
     QErr
     m
     ([TableMeta ('Postgres pgKind)], [FunctionMeta ('Postgres pgKind)])
forall (pgKind :: PostgresKind) (m :: * -> *).
(ToMetadataFetchQuery pgKind, FetchTableMetadata pgKind,
 FetchFunctionMetadata pgKind, BackendMetadata ('Postgres pgKind),
 MonadTx m) =>
TableCache ('Postgres pgKind)
-> HashSet (TableName ('Postgres pgKind))
-> HashSet (FunctionName ('Postgres pgKind))
-> m ([TableMeta ('Postgres pgKind)],
      [FunctionMeta ('Postgres pgKind)])
fetchTablesFunctionsMetadata TableCache ('Postgres pgKind)
tableCache)
          -- Fetch names of tables and functions using OIDs which also contains renamed items
          ((HashSet QualifiedTable, HashSet QualifiedFunction)
 -> TxET
      QErr
      m
      ([TableMeta ('Postgres pgKind)],
       [FunctionMeta ('Postgres pgKind)]))
-> TxET QErr m (HashSet QualifiedTable, HashSet QualifiedFunction)
-> TxET
     QErr
     m
     ([TableMeta ('Postgres pgKind)], [FunctionMeta ('Postgres pgKind)])
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< HashSet OID
-> HashSet OID
-> TxET
     QErr
     m
     (HashSet (TableName ('Postgres Any)),
      HashSet (FunctionName ('Postgres Any)))
forall (m :: * -> *) (pgKind :: PostgresKind).
MonadIO m =>
HashSet OID
-> HashSet OID
-> TxET
     QErr
     m
     (HashSet (TableName ('Postgres pgKind)),
      HashSet (FunctionName ('Postgres pgKind)))
fetchTablesFunctionsFromOids HashSet OID
tableOids HashSet OID
functionOids

      -- Calculate the tables diff (dropped & altered tables)
      let tablesDiff :: TablesDiff ('Postgres pgKind)
tablesDiff = [TableMeta ('Postgres pgKind)]
-> [TableMeta ('Postgres pgKind)] -> TablesDiff ('Postgres pgKind)
forall (b :: BackendType).
Backend b =>
[TableMeta b] -> [TableMeta b] -> TablesDiff b
Diff.getTablesDiff [TableMeta ('Postgres pgKind)]
preTxTablesMeta [TableMeta ('Postgres pgKind)]
postTxTablesMeta
          -- Calculate the functions diff. For calculating diff for functions, only consider
          -- query/mutation functions and exclude functions underpinning computed fields.
          -- Computed field functions are being processed under each table diff.
          -- See @'getTablesDiff' and @'Diff.processTablesDiff'
          excludeComputedFieldFunctions :: [FunctionMeta ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
excludeComputedFieldFunctions = (FunctionMeta ('Postgres pgKind) -> Bool)
-> [FunctionMeta ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
forall a. (a -> Bool) -> [a] -> [a]
filter ((FunctionName ('Postgres pgKind)
-> FunctionCache ('Postgres pgKind) -> Bool
forall k a. (Eq k, Hashable k) => k -> HashMap k a -> Bool
`HashMap.member` FunctionCache ('Postgres pgKind)
functionCache) (QualifiedFunction -> Bool)
-> (FunctionMeta ('Postgres pgKind) -> QualifiedFunction)
-> FunctionMeta ('Postgres pgKind)
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FunctionMeta ('Postgres pgKind) -> FunctionName ('Postgres pgKind)
FunctionMeta ('Postgres pgKind) -> QualifiedFunction
forall (b :: BackendType). FunctionMeta b -> FunctionName b
Diff.fmFunction)
          functionsDiff :: FunctionsDiff ('Postgres pgKind)
functionsDiff =
            [FunctionMeta ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
-> FunctionsDiff ('Postgres pgKind)
forall (b :: BackendType).
[FunctionMeta b] -> [FunctionMeta b] -> FunctionsDiff b
Diff.getFunctionsDiff
              ([FunctionMeta ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
excludeComputedFieldFunctions [FunctionMeta ('Postgres pgKind)]
preTxFunctionsMeta)
              ([FunctionMeta ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
excludeComputedFieldFunctions [FunctionMeta ('Postgres pgKind)]
postTxFunctionMeta)

      [FunctionName ('Postgres pgKind)] -> TxET QErr m ()
forall (n :: * -> *).
MonadError QErr n =>
[FunctionName ('Postgres pgKind)] -> n ()
dontAllowFunctionOverloading
        ([FunctionName ('Postgres pgKind)] -> TxET QErr m ())
-> [FunctionName ('Postgres pgKind)] -> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ [FunctionName ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
-> [FunctionName ('Postgres pgKind)]
forall (b :: BackendType).
Backend b =>
[FunctionName b] -> [FunctionMeta b] -> [FunctionName b]
Diff.getOverloadedFunctions
          (HashMap QualifiedFunction (FunctionInfo ('Postgres pgKind))
-> [QualifiedFunction]
forall k v. HashMap k v -> [k]
HashMap.keys FunctionCache ('Postgres pgKind)
HashMap QualifiedFunction (FunctionInfo ('Postgres pgKind))
functionCache)
          ([FunctionMeta ('Postgres pgKind)]
-> [FunctionMeta ('Postgres pgKind)]
excludeComputedFieldFunctions [FunctionMeta ('Postgres pgKind)]
postTxFunctionMeta)

      -- Update metadata with schema change caused by @'tx'
      MetadataModifier
metadataUpdater <- WriterT MetadataModifier (TxET QErr m) ()
-> TxET QErr m MetadataModifier
forall (m :: * -> *) w a. Monad m => WriterT w m a -> m w
execWriterT do
        -- Collect indirect dependencies of altered tables
        [SchemaObjId]
tableIndirectDeps <- SourceName
-> TablesDiff ('Postgres pgKind)
-> WriterT MetadataModifier (TxET QErr m) [SchemaObjId]
forall (b :: BackendType) (m :: * -> *).
(QErrM m, CacheRM m, Backend b) =>
SourceName -> TablesDiff b -> m [SchemaObjId]
Diff.getIndirectDependenciesFromTableDiff SourceName
source TablesDiff ('Postgres pgKind)
tablesDiff

        -- If table indirect dependencies exist and cascading is not enabled then report an exception
        Bool
-> WriterT MetadataModifier (TxET QErr m) ()
-> WriterT MetadataModifier (TxET QErr m) ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless ([SchemaObjId] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [SchemaObjId]
tableIndirectDeps Bool -> Bool -> Bool
|| Bool
cascadeDependencies) (WriterT MetadataModifier (TxET QErr m) ()
 -> WriterT MetadataModifier (TxET QErr m) ())
-> WriterT MetadataModifier (TxET QErr m) ()
-> WriterT MetadataModifier (TxET QErr m) ()
forall a b. (a -> b) -> a -> b
$ [SchemaObjId] -> WriterT MetadataModifier (TxET QErr m) ()
forall (m :: * -> *). MonadError QErr m => [SchemaObjId] -> m ()
reportDependentObjectsExist [SchemaObjId]
tableIndirectDeps

        -- Purge all the table dependents
        (SchemaObjId -> WriterT MetadataModifier (TxET QErr m) ())
-> [SchemaObjId] -> WriterT MetadataModifier (TxET QErr m) ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
(a -> f b) -> t a -> f ()
traverse_ SchemaObjId -> WriterT MetadataModifier (TxET QErr m) ()
forall (m :: * -> *).
MonadError QErr m =>
SchemaObjId -> WriterT MetadataModifier m ()
purgeSourceAndSchemaDependencies [SchemaObjId]
tableIndirectDeps

        -- Collect function names from purged table dependencies
        let purgedFunctions :: [FunctionName ('Postgres pgKind)]
purgedFunctions = [SchemaObjId] -> [FunctionName ('Postgres pgKind)]
collectFunctionsInDeps [SchemaObjId]
tableIndirectDeps
            Diff.FunctionsDiff [FunctionName ('Postgres pgKind)]
droppedFunctions [(FunctionName ('Postgres pgKind), FunctionVolatility)]
alteredFunctions = FunctionsDiff ('Postgres pgKind)
functionsDiff

        -- Drop functions in metadata. Exclude functions that were already dropped as part of table indirect dependencies
        [FunctionName ('Postgres pgKind)]
-> WriterT MetadataModifier (TxET QErr m) ()
forall (n :: * -> *).
Monad n =>
[FunctionName ('Postgres pgKind)] -> WriterT MetadataModifier n ()
purgeFunctionsFromMetadata ([FunctionName ('Postgres pgKind)]
 -> WriterT MetadataModifier (TxET QErr m) ())
-> [FunctionName ('Postgres pgKind)]
-> WriterT MetadataModifier (TxET QErr m) ()
forall a b. (a -> b) -> a -> b
$ [FunctionName ('Postgres pgKind)]
[QualifiedFunction]
droppedFunctions [QualifiedFunction] -> [QualifiedFunction] -> [QualifiedFunction]
forall a. Eq a => [a] -> [a] -> [a]
\\ [FunctionName ('Postgres pgKind)]
[QualifiedFunction]
purgedFunctions

        -- If any function type is altered to VOLATILE then raise an exception
        [(FunctionName ('Postgres pgKind), FunctionVolatility)]
-> WriterT MetadataModifier (TxET QErr m) ()
forall (n :: * -> *).
MonadError QErr n =>
[(FunctionName ('Postgres pgKind), FunctionVolatility)] -> n ()
dontAllowFunctionAlteredVolatile [(FunctionName ('Postgres pgKind), FunctionVolatility)]
alteredFunctions

        -- Propagate table changes to metadata
        SourceName
-> TableCache ('Postgres pgKind)
-> TablesDiff ('Postgres pgKind)
-> WriterT MetadataModifier (TxET QErr m) ()
forall (b :: BackendType) (m :: * -> *).
(MonadError QErr m, CacheRM m, MonadWriter MetadataModifier m,
 BackendMetadata b) =>
SourceName -> TableCache b -> TablesDiff b -> m ()
Diff.processTablesDiff SourceName
source TableCache ('Postgres pgKind)
tableCache TablesDiff ('Postgres pgKind)
tablesDiff

      (a, MetadataModifier) -> TxET QErr m (a, MetadataModifier)
forall a. a -> TxET QErr m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (a
txResult, MetadataModifier
metadataUpdater)
  where
    dontAllowFunctionOverloading ::
      (MonadError QErr n) =>
      [FunctionName ('Postgres pgKind)] ->
      n ()
    dontAllowFunctionOverloading :: forall (n :: * -> *).
MonadError QErr n =>
[FunctionName ('Postgres pgKind)] -> n ()
dontAllowFunctionOverloading [FunctionName ('Postgres pgKind)]
overloadedFunctions =
      Bool -> n () -> n ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
unless ([QualifiedFunction] -> Bool
forall a. [a] -> Bool
forall (t :: * -> *) a. Foldable t => t a -> Bool
null [FunctionName ('Postgres pgKind)]
[QualifiedFunction]
overloadedFunctions)
        (n () -> n ()) -> n () -> n ()
forall a b. (a -> b) -> a -> b
$ Code -> Text -> n ()
forall (m :: * -> *) a. QErrM m => Code -> Text -> m a
throw400 Code
NotSupported
        (Text -> n ()) -> Text -> n ()
forall a b. (a -> b) -> a -> b
$ Text
"the following tracked function(s) cannot be overloaded: "
        Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> [QualifiedFunction] -> Text
forall t (f :: * -> *). (ToTxt t, Foldable f) => f t -> Text
commaSeparated [FunctionName ('Postgres pgKind)]
[QualifiedFunction]
overloadedFunctions

    dontAllowFunctionAlteredVolatile ::
      (MonadError QErr n) =>
      [(FunctionName ('Postgres pgKind), FunctionVolatility)] ->
      n ()
    dontAllowFunctionAlteredVolatile :: forall (n :: * -> *).
MonadError QErr n =>
[(FunctionName ('Postgres pgKind), FunctionVolatility)] -> n ()
dontAllowFunctionAlteredVolatile [(FunctionName ('Postgres pgKind), FunctionVolatility)]
alteredFunctions =
      [(QualifiedFunction, FunctionVolatility)]
-> ((QualifiedFunction, FunctionVolatility) -> n ()) -> n ()
forall (t :: * -> *) (m :: * -> *) a b.
(Foldable t, Monad m) =>
t a -> (a -> m b) -> m ()
forM_ [(FunctionName ('Postgres pgKind), FunctionVolatility)]
[(QualifiedFunction, FunctionVolatility)]
alteredFunctions (((QualifiedFunction, FunctionVolatility) -> n ()) -> n ())
-> ((QualifiedFunction, FunctionVolatility) -> n ()) -> n ()
forall a b. (a -> b) -> a -> b
$ \(QualifiedFunction
qf, FunctionVolatility
newTy) -> do
        Bool -> n () -> n ()
forall (f :: * -> *). Applicative f => Bool -> f () -> f ()
when (FunctionVolatility
newTy FunctionVolatility -> FunctionVolatility -> Bool
forall a. Eq a => a -> a -> Bool
== FunctionVolatility
FTVOLATILE)
          (n () -> n ()) -> n () -> n ()
forall a b. (a -> b) -> a -> b
$ Code -> Text -> n ()
forall (m :: * -> *) a. QErrM m => Code -> Text -> m a
throw400 Code
NotSupported
          (Text -> n ()) -> Text -> n ()
forall a b. (a -> b) -> a -> b
$ Text
"type of function "
          Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> QualifiedFunction
qf
          QualifiedFunction -> Text -> Text
forall t. ToTxt t => t -> Text -> Text
<<> Text
" is altered to \"VOLATILE\" which is not supported now"

    purgeFunctionsFromMetadata ::
      (Monad n) =>
      [FunctionName ('Postgres pgKind)] ->
      WriterT MetadataModifier n ()
    purgeFunctionsFromMetadata :: forall (n :: * -> *).
Monad n =>
[FunctionName ('Postgres pgKind)] -> WriterT MetadataModifier n ()
purgeFunctionsFromMetadata [FunctionName ('Postgres pgKind)]
functions =
      [QualifiedFunction]
-> (QualifiedFunction -> WriterT MetadataModifier n ())
-> WriterT MetadataModifier n ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ [FunctionName ('Postgres pgKind)]
[QualifiedFunction]
functions ((QualifiedFunction -> WriterT MetadataModifier n ())
 -> WriterT MetadataModifier n ())
-> (QualifiedFunction -> WriterT MetadataModifier n ())
-> WriterT MetadataModifier n ()
forall a b. (a -> b) -> a -> b
$ MetadataModifier -> WriterT MetadataModifier n ()
forall w (m :: * -> *). MonadWriter w m => w -> m ()
tell (MetadataModifier -> WriterT MetadataModifier n ())
-> (QualifiedFunction -> MetadataModifier)
-> QualifiedFunction
-> WriterT MetadataModifier n ()
forall b c a. (b -> c) -> (a -> b) -> a -> c
. forall (b :: BackendType).
Backend b =>
SourceName -> FunctionName b -> MetadataModifier
dropFunctionInMetadata @('Postgres pgKind) SourceName
source

    collectFunctionsInDeps :: [SchemaObjId] -> [FunctionName ('Postgres pgKind)]
    collectFunctionsInDeps :: [SchemaObjId] -> [FunctionName ('Postgres pgKind)]
collectFunctionsInDeps [SchemaObjId]
deps =
      ((SchemaObjId -> Maybe QualifiedFunction)
 -> [SchemaObjId] -> [QualifiedFunction])
-> [SchemaObjId]
-> (SchemaObjId -> Maybe QualifiedFunction)
-> [QualifiedFunction]
forall a b c. (a -> b -> c) -> b -> a -> c
flip (SchemaObjId -> Maybe QualifiedFunction)
-> [SchemaObjId] -> [QualifiedFunction]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe [SchemaObjId]
deps \case
        SOSourceObj SourceName
_ AnyBackend SourceObjId
objectID
          | Just (SOIFunction FunctionName ('Postgres pgKind)
qf) <- forall (b :: BackendType) (i :: BackendType -> *).
HasTag b =>
AnyBackend i -> Maybe (i b)
AB.unpackAnyBackend @('Postgres pgKind) AnyBackend SourceObjId
objectID ->
              QualifiedFunction -> Maybe QualifiedFunction
forall a. a -> Maybe a
Just FunctionName ('Postgres pgKind)
QualifiedFunction
qf
        SchemaObjId
_ -> Maybe QualifiedFunction
forall a. Maybe a
Nothing

-- | Fetch list of tables and functions with provided oids
fetchTablesFunctionsFromOids ::
  (MonadIO m) =>
  HashSet OID ->
  HashSet OID ->
  PG.TxET
    QErr
    m
    ( HS.HashSet (TableName ('Postgres pgKind)),
      HS.HashSet (FunctionName ('Postgres pgKind))
    )
fetchTablesFunctionsFromOids :: forall (m :: * -> *) (pgKind :: PostgresKind).
MonadIO m =>
HashSet OID
-> HashSet OID
-> TxET
     QErr
     m
     (HashSet (TableName ('Postgres pgKind)),
      HashSet (FunctionName ('Postgres pgKind)))
fetchTablesFunctionsFromOids HashSet OID
tableOids HashSet OID
functionOids =
  ((ViaJSON (HashSet QualifiedTable) -> HashSet QualifiedTable
forall a. ViaJSON a -> a
PG.getViaJSON (ViaJSON (HashSet QualifiedTable) -> HashSet QualifiedTable)
-> (ViaJSON (HashSet QualifiedFunction)
    -> HashSet QualifiedFunction)
-> (ViaJSON (HashSet QualifiedTable),
    ViaJSON (HashSet QualifiedFunction))
-> (HashSet QualifiedTable, HashSet QualifiedFunction)
forall b c b' c'. (b -> c) -> (b' -> c') -> (b, b') -> (c, c')
forall (a :: * -> * -> *) b c b' c'.
Arrow a =>
a b c -> a b' c' -> a (b, b') (c, c')
*** ViaJSON (HashSet QualifiedFunction) -> HashSet QualifiedFunction
forall a. ViaJSON a -> a
PG.getViaJSON) ((ViaJSON (HashSet QualifiedTable),
  ViaJSON (HashSet QualifiedFunction))
 -> (HashSet QualifiedTable, HashSet QualifiedFunction))
-> (SingleRow
      (ViaJSON (HashSet QualifiedTable),
       ViaJSON (HashSet QualifiedFunction))
    -> (ViaJSON (HashSet QualifiedTable),
        ViaJSON (HashSet QualifiedFunction)))
-> SingleRow
     (ViaJSON (HashSet QualifiedTable),
      ViaJSON (HashSet QualifiedFunction))
-> (HashSet QualifiedTable, HashSet QualifiedFunction)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SingleRow
  (ViaJSON (HashSet QualifiedTable),
   ViaJSON (HashSet QualifiedFunction))
-> (ViaJSON (HashSet QualifiedTable),
    ViaJSON (HashSet QualifiedFunction))
forall a. SingleRow a -> a
PG.getRow)
    (SingleRow
   (ViaJSON (HashSet QualifiedTable),
    ViaJSON (HashSet QualifiedFunction))
 -> (HashSet QualifiedTable, HashSet QualifiedFunction))
-> TxET
     QErr
     m
     (SingleRow
        (ViaJSON (HashSet QualifiedTable),
         ViaJSON (HashSet QualifiedFunction)))
-> TxET QErr m (HashSet QualifiedTable, HashSet QualifiedFunction)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (PGTxErr -> QErr)
-> Query
-> (ViaJSON [Value], ViaJSON [Value])
-> Bool
-> TxET
     QErr
     m
     (SingleRow
        (ViaJSON (HashSet QualifiedTable),
         ViaJSON (HashSet QualifiedFunction)))
forall (m :: * -> *) a r e.
(MonadIO m, FromRes a, ToPrepArgs r) =>
(PGTxErr -> e) -> Query -> r -> Bool -> TxET e m a
PG.withQE
      PGTxErr -> QErr
defaultTxErrorHandler
      [PG.sql|
    SELECT
      COALESCE(
        ( SELECT
            json_agg(
              row_to_json(
                (
                  SELECT e
                    FROM ( SELECT "table".relname AS "name",
                                  "schema".nspname AS "schema"
                    ) AS e
                )
              )
            ) AS "item"
            FROM jsonb_to_recordset($1::jsonb) AS oid_table("oid" int)
                 JOIN pg_catalog.pg_class "table" ON ("table".oid = "oid_table".oid)
                 JOIN pg_catalog.pg_namespace "schema" ON ("schema".oid = "table".relnamespace)
        ),
        '[]'
      ) AS "tables",

      COALESCE(
        ( SELECT
            json_agg(
              row_to_json(
                (
                  SELECT e
                    FROM ( SELECT "function".proname AS "name",
                                  "schema".nspname AS "schema"
                    ) AS e
                )
              )
            ) AS "item"
            FROM jsonb_to_recordset($2::jsonb) AS oid_table("oid" int)
                 JOIN pg_catalog.pg_proc "function" ON ("function".oid = "oid_table".oid)
                 JOIN pg_catalog.pg_namespace "schema" ON ("schema".oid = "function".pronamespace)
        ),
        '[]'
      ) AS "functions"
  |]
      ([Value] -> ViaJSON [Value]
forall a. a -> ViaJSON a
PG.ViaJSON ([Value] -> ViaJSON [Value]) -> [Value] -> ViaJSON [Value]
forall a b. (a -> b) -> a -> b
$ (OID -> Value) -> [OID] -> [Value]
forall a b. (a -> b) -> [a] -> [b]
map OID -> Value
forall a. ToJSON a => a -> Value
mkOidObject ([OID] -> [Value]) -> [OID] -> [Value]
forall a b. (a -> b) -> a -> b
$ HashSet OID -> [OID]
forall a. HashSet a -> [a]
HS.toList HashSet OID
tableOids, [Value] -> ViaJSON [Value]
forall a. a -> ViaJSON a
PG.ViaJSON ([Value] -> ViaJSON [Value]) -> [Value] -> ViaJSON [Value]
forall a b. (a -> b) -> a -> b
$ (OID -> Value) -> [OID] -> [Value]
forall a b. (a -> b) -> [a] -> [b]
map OID -> Value
forall a. ToJSON a => a -> Value
mkOidObject ([OID] -> [Value]) -> [OID] -> [Value]
forall a b. (a -> b) -> a -> b
$ HashSet OID -> [OID]
forall a. HashSet a -> [a]
HS.toList (HashSet OID -> [OID]) -> HashSet OID -> [OID]
forall a b. (a -> b) -> a -> b
$ HashSet OID
functionOids)
      Bool
True
  where
    mkOidObject :: v -> Value
mkOidObject v
oid = [Pair] -> Value
object [Key
"oid" Key -> v -> Pair
forall kv v. (KeyValue kv, ToJSON v) => Key -> v -> kv
forall v. ToJSON v => Key -> v -> Pair
.= v
oid]

------ helpers ------------

getComputedFields :: TableInfo ('Postgres pgKind) -> [ComputedFieldInfo ('Postgres pgKind)]
getComputedFields :: forall (pgKind :: PostgresKind).
TableInfo ('Postgres pgKind)
-> [ComputedFieldInfo ('Postgres pgKind)]
getComputedFields = FieldInfoMap (FieldInfo ('Postgres pgKind))
-> [ComputedFieldInfo ('Postgres pgKind)]
forall (backend :: BackendType).
FieldInfoMap (FieldInfo backend) -> [ComputedFieldInfo backend]
getComputedFieldInfos (FieldInfoMap (FieldInfo ('Postgres pgKind))
 -> [ComputedFieldInfo ('Postgres pgKind)])
-> (TableInfo ('Postgres pgKind)
    -> FieldInfoMap (FieldInfo ('Postgres pgKind)))
-> TableInfo ('Postgres pgKind)
-> [ComputedFieldInfo ('Postgres pgKind)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TableCoreInfoG
  ('Postgres pgKind)
  (FieldInfo ('Postgres pgKind))
  (ColumnInfo ('Postgres pgKind))
-> FieldInfoMap (FieldInfo ('Postgres pgKind))
forall (b :: BackendType) field primaryKeyColumn.
TableCoreInfoG b field primaryKeyColumn -> FieldInfoMap field
_tciFieldInfoMap (TableCoreInfoG
   ('Postgres pgKind)
   (FieldInfo ('Postgres pgKind))
   (ColumnInfo ('Postgres pgKind))
 -> FieldInfoMap (FieldInfo ('Postgres pgKind)))
-> (TableInfo ('Postgres pgKind)
    -> TableCoreInfoG
         ('Postgres pgKind)
         (FieldInfo ('Postgres pgKind))
         (ColumnInfo ('Postgres pgKind)))
-> TableInfo ('Postgres pgKind)
-> FieldInfoMap (FieldInfo ('Postgres pgKind))
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TableInfo ('Postgres pgKind)
-> TableCoreInfoG
     ('Postgres pgKind)
     (FieldInfo ('Postgres pgKind))
     (ColumnInfo ('Postgres pgKind))
forall (b :: BackendType). TableInfo b -> TableCoreInfo b
_tiCoreInfo

getComputedFieldFunctions :: TableInfo ('Postgres pgKind) -> HashSet (FunctionName ('Postgres pgKind))
getComputedFieldFunctions :: forall (pgKind :: PostgresKind).
TableInfo ('Postgres pgKind)
-> HashSet (FunctionName ('Postgres pgKind))
getComputedFieldFunctions = [QualifiedFunction] -> HashSet QualifiedFunction
forall a. (Eq a, Hashable a) => [a] -> HashSet a
HS.fromList ([QualifiedFunction] -> HashSet QualifiedFunction)
-> (TableInfo ('Postgres pgKind) -> [QualifiedFunction])
-> TableInfo ('Postgres pgKind)
-> HashSet QualifiedFunction
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ComputedFieldInfo ('Postgres pgKind) -> QualifiedFunction)
-> [ComputedFieldInfo ('Postgres pgKind)] -> [QualifiedFunction]
forall a b. (a -> b) -> [a] -> [b]
map (ComputedFieldFunction ('Postgres pgKind)
-> FunctionName ('Postgres pgKind)
ComputedFieldFunction ('Postgres pgKind) -> QualifiedFunction
forall (b :: BackendType).
ComputedFieldFunction b -> FunctionName b
_cffName (ComputedFieldFunction ('Postgres pgKind) -> QualifiedFunction)
-> (ComputedFieldInfo ('Postgres pgKind)
    -> ComputedFieldFunction ('Postgres pgKind))
-> ComputedFieldInfo ('Postgres pgKind)
-> QualifiedFunction
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ComputedFieldInfo ('Postgres pgKind)
-> ComputedFieldFunction ('Postgres pgKind)
forall (b :: BackendType).
ComputedFieldInfo b -> ComputedFieldFunction b
_cfiFunction) ([ComputedFieldInfo ('Postgres pgKind)] -> [QualifiedFunction])
-> (TableInfo ('Postgres pgKind)
    -> [ComputedFieldInfo ('Postgres pgKind)])
-> TableInfo ('Postgres pgKind)
-> [QualifiedFunction]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TableInfo ('Postgres pgKind)
-> [ComputedFieldInfo ('Postgres pgKind)]
forall (pgKind :: PostgresKind).
TableInfo ('Postgres pgKind)
-> [ComputedFieldInfo ('Postgres pgKind)]
getComputedFields