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

-- | Postgres DDL Source
--
-- A Source is a connected database. One can have multiple sources of the same
-- kind (e.g. Postgres).
--
-- This module provides ways to fetch, update, and deal with table and function
-- metadata and hdb_catalog migrations for a Postgres Source.
--
--    NOTE: Please have a look at the `server/documentation/migration-guidelines.md`
--       before adding any new migration if you haven't already looked at it.
module Hasura.Backends.Postgres.DDL.Source
  ( ToMetadataFetchQuery,
    FetchTableMetadata (..),
    FetchFunctionMetadata (..),
    prepareCatalog,
    postDropSourceHook,
    resolveDatabaseMetadata,
    resolveSourceConfig,
    logPGSourceCatalogMigrationLockedQueries,
    -- naughty exports, forgive me padre
    pgFetchTableMetadata,
  )
where

import Control.Concurrent.Extended (sleep)
import Control.Monad.Trans.Control (MonadBaseControl)
import Data.Aeson (FromJSON (..), ToJSON (..), genericParseJSON, genericToEncoding, genericToJSON, toJSON)
import Data.Environment qualified as Env
import Data.FileEmbed (makeRelativeToProject)
import Data.HashMap.Strict.Extended qualified as HashMap
import Data.HashMap.Strict.InsOrd qualified as InsOrdHashMap
import Data.HashSet qualified as Set
import Data.List.Extended qualified as LE
import Data.List.NonEmpty qualified as NE
import Data.Time.Clock (UTCTime, getCurrentTime)
import Database.PG.Query qualified as PG
import Hasura.Backends.Postgres.Connection
import Hasura.Backends.Postgres.DDL.EventTrigger (dropTriggerQ)
import Hasura.Backends.Postgres.DDL.Source.Version
import Hasura.Backends.Postgres.SQL.Types hiding (FunctionName)
import Hasura.Backends.Postgres.Types.ComputedField
import Hasura.Base.Error
import Hasura.Function.Cache
import Hasura.Logging
import Hasura.Prelude
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.BackendType
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.EventTrigger (RecreateEventTriggers (..))
import Hasura.RQL.Types.Metadata (SourceMetadata (..), _cfmDefinition)
import Hasura.RQL.Types.Source
import Hasura.Server.Migrate.Internal
import Hasura.Server.Migrate.Version qualified as Version
import Hasura.Table.Cache
import Hasura.Table.Metadata (TableMetadata (..))
import Language.Haskell.TH.Lib qualified as TH
import Language.Haskell.TH.Syntax qualified as TH

-- | We differentiate the handling of metadata between Citus, Cockroach and Vanilla
-- Postgres because Citus imposes limitations on the types of joins that it
-- permits, which then limits the types of relations that we can track.
class ToMetadataFetchQuery (pgKind :: PostgresKind) where
  tableMetadata :: PG.Query

instance ToMetadataFetchQuery 'Vanilla where
  tableMetadata :: Query
tableMetadata = $(makeRelativeToProject "src-rsr/pg_table_metadata.sql" >>= PG.sqlFromFile)

instance ToMetadataFetchQuery 'Citus where
  tableMetadata :: Query
tableMetadata = $(makeRelativeToProject "src-rsr/citus_table_metadata.sql" >>= PG.sqlFromFile)

instance ToMetadataFetchQuery 'Cockroach where
  tableMetadata :: Query
tableMetadata = $(makeRelativeToProject "src-rsr/cockroach_table_metadata.sql" >>= PG.sqlFromFile)

resolveSourceConfig ::
  (MonadIO m, MonadResolveSource m) =>
  SourceName ->
  PostgresConnConfiguration ->
  BackendSourceKind ('Postgres pgKind) ->
  BackendConfig ('Postgres pgKind) ->
  Env.Environment ->
  manager ->
  m (Either QErr (SourceConfig ('Postgres pgKind)))
resolveSourceConfig :: forall (m :: * -> *) (pgKind :: PostgresKind) manager.
(MonadIO m, MonadResolveSource m) =>
SourceName
-> PostgresConnConfiguration
-> BackendSourceKind ('Postgres pgKind)
-> BackendConfig ('Postgres pgKind)
-> Environment
-> manager
-> m (Either QErr (SourceConfig ('Postgres pgKind)))
resolveSourceConfig SourceName
name PostgresConnConfiguration
config BackendSourceKind ('Postgres pgKind)
_backendKind BackendConfig ('Postgres pgKind)
_backendConfig Environment
env manager
_manager = ExceptT QErr m PGSourceConfig -> m (Either QErr PGSourceConfig)
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT do
  Environment
-> SourceName
-> PostgresConnConfiguration
-> IO (Either QErr PGSourceConfig)
sourceResolver <- ExceptT QErr m (SourceResolver ('Postgres 'Vanilla))
ExceptT
  QErr
  m
  (Environment
   -> SourceName
   -> PostgresConnConfiguration
   -> IO (Either QErr PGSourceConfig))
forall (m :: * -> *).
MonadResolveSource m =>
m (SourceResolver ('Postgres 'Vanilla))
getPGSourceResolver
  ExceptT QErr m (Either QErr PGSourceConfig)
-> ExceptT QErr m PGSourceConfig
forall e (m :: * -> *) a. MonadError e m => m (Either e a) -> m a
liftEitherM (ExceptT QErr m (Either QErr PGSourceConfig)
 -> ExceptT QErr m PGSourceConfig)
-> ExceptT QErr m (Either QErr PGSourceConfig)
-> ExceptT QErr m PGSourceConfig
forall a b. (a -> b) -> a -> b
$ IO (Either QErr PGSourceConfig)
-> ExceptT QErr m (Either QErr PGSourceConfig)
forall a. IO a -> ExceptT QErr m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Either QErr PGSourceConfig)
 -> ExceptT QErr m (Either QErr PGSourceConfig))
-> IO (Either QErr PGSourceConfig)
-> ExceptT QErr m (Either QErr PGSourceConfig)
forall a b. (a -> b) -> a -> b
$ Environment
-> SourceName
-> PostgresConnConfiguration
-> IO (Either QErr PGSourceConfig)
sourceResolver Environment
env SourceName
name PostgresConnConfiguration
config

-- | 'PGSourceLockQuery' is a data type which represents the contents of a single object of the
--   locked queries which are queried from the `pg_stat_activity`. See `logPGSourceCatalogMigrationLockedQueries`.
data PGSourceLockQuery = PGSourceLockQuery
  { PGSourceLockQuery -> Text
_psqaQuery :: Text,
    PGSourceLockQuery -> Maybe Bool
_psqaLockGranted :: Maybe Bool,
    PGSourceLockQuery -> Text
_psqaLockMode :: Text,
    PGSourceLockQuery -> UTCTime
_psqaTransactionStartTime :: UTCTime,
    PGSourceLockQuery -> UTCTime
_psqaQueryStartTime :: UTCTime,
    PGSourceLockQuery -> Text
_psqaWaitEventType :: Text,
    PGSourceLockQuery -> Text
_psqaBlockingQuery :: Text
  }
  deriving stock ((forall x. PGSourceLockQuery -> Rep PGSourceLockQuery x)
-> (forall x. Rep PGSourceLockQuery x -> PGSourceLockQuery)
-> Generic PGSourceLockQuery
forall x. Rep PGSourceLockQuery x -> PGSourceLockQuery
forall x. PGSourceLockQuery -> Rep PGSourceLockQuery x
forall a.
(forall x. a -> Rep a x) -> (forall x. Rep a x -> a) -> Generic a
$cfrom :: forall x. PGSourceLockQuery -> Rep PGSourceLockQuery x
from :: forall x. PGSourceLockQuery -> Rep PGSourceLockQuery x
$cto :: forall x. Rep PGSourceLockQuery x -> PGSourceLockQuery
to :: forall x. Rep PGSourceLockQuery x -> PGSourceLockQuery
Generic)

instance FromJSON PGSourceLockQuery where
  parseJSON :: Value -> Parser PGSourceLockQuery
parseJSON = Options -> Value -> Parser PGSourceLockQuery
forall a.
(Generic a, GFromJSON Zero (Rep a)) =>
Options -> Value -> Parser a
genericParseJSON Options
hasuraJSON

instance ToJSON PGSourceLockQuery where
  toJSON :: PGSourceLockQuery -> Value
toJSON = Options -> PGSourceLockQuery -> Value
forall a.
(Generic a, GToJSON' Value Zero (Rep a)) =>
Options -> a -> Value
genericToJSON Options
hasuraJSON
  toEncoding :: PGSourceLockQuery -> Encoding
toEncoding = Options -> PGSourceLockQuery -> Encoding
forall a.
(Generic a, GToJSON' Encoding Zero (Rep a)) =>
Options -> a -> Encoding
genericToEncoding Options
hasuraJSON

instance ToEngineLog [PGSourceLockQuery] Hasura where
  toEngineLog :: [PGSourceLockQuery] -> (LogLevel, EngineLogType Hasura, Value)
toEngineLog [PGSourceLockQuery]
resp = (LogLevel
LevelInfo, EngineLogType Hasura
sourceCatalogMigrationLogType, [PGSourceLockQuery] -> Value
forall a. ToJSON a => a -> Value
toJSON [PGSourceLockQuery]
resp)

newtype PGSourceLockQueryError = PGSourceLockQueryError QErr
  deriving ([PGSourceLockQueryError] -> Value
[PGSourceLockQueryError] -> Encoding
PGSourceLockQueryError -> Value
PGSourceLockQueryError -> Encoding
(PGSourceLockQueryError -> Value)
-> (PGSourceLockQueryError -> Encoding)
-> ([PGSourceLockQueryError] -> Value)
-> ([PGSourceLockQueryError] -> Encoding)
-> ToJSON PGSourceLockQueryError
forall a.
(a -> Value)
-> (a -> Encoding)
-> ([a] -> Value)
-> ([a] -> Encoding)
-> ToJSON a
$ctoJSON :: PGSourceLockQueryError -> Value
toJSON :: PGSourceLockQueryError -> Value
$ctoEncoding :: PGSourceLockQueryError -> Encoding
toEncoding :: PGSourceLockQueryError -> Encoding
$ctoJSONList :: [PGSourceLockQueryError] -> Value
toJSONList :: [PGSourceLockQueryError] -> Value
$ctoEncodingList :: [PGSourceLockQueryError] -> Encoding
toEncodingList :: [PGSourceLockQueryError] -> Encoding
ToJSON)

instance ToEngineLog PGSourceLockQueryError Hasura where
  toEngineLog :: PGSourceLockQueryError -> (LogLevel, EngineLogType Hasura, Value)
toEngineLog PGSourceLockQueryError
resp = (LogLevel
LevelError, EngineLogType Hasura
sourceCatalogMigrationLogType, PGSourceLockQueryError -> Value
forall a. ToJSON a => a -> Value
toJSON PGSourceLockQueryError
resp)

-- | 'logPGSourceCatalogMigrationLockedQueries' as the name suggests logs
--   the queries which are blocking in the database. This function is called
--   asynchronously from `initCatalogIfNeeded` while the source catalog is being
--   migrated.
--   NOTE: When there are no locking queries present in the database, nothing will be logged.
logPGSourceCatalogMigrationLockedQueries ::
  (MonadIO m) =>
  Logger Hasura ->
  PGSourceConfig ->
  m Void
logPGSourceCatalogMigrationLockedQueries :: forall (m :: * -> *).
MonadIO m =>
Logger Hasura -> PGSourceConfig -> m Void
logPGSourceCatalogMigrationLockedQueries Logger Hasura
logger PGSourceConfig
sourceConfig = m () -> m Void
forall (f :: * -> *) a b. Applicative f => f a -> f b
forever (m () -> m Void) -> m () -> m Void
forall a b. (a -> b) -> a -> b
$ do
  Either QErr (Maybe [PGSourceLockQuery])
dbStats <- IO (Either QErr (Maybe [PGSourceLockQuery]))
-> m (Either QErr (Maybe [PGSourceLockQuery]))
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Either QErr (Maybe [PGSourceLockQuery]))
 -> m (Either QErr (Maybe [PGSourceLockQuery])))
-> IO (Either QErr (Maybe [PGSourceLockQuery]))
-> m (Either QErr (Maybe [PGSourceLockQuery]))
forall a b. (a -> b) -> a -> b
$ PGSourceConfig
-> TxET QErr IO (Maybe [PGSourceLockQuery])
-> IO (Either QErr (Maybe [PGSourceLockQuery]))
forall (m :: * -> *) a.
(MonadIO m, MonadBaseControl IO m) =>
PGSourceConfig -> TxET QErr m a -> m (Either QErr a)
runPgSourceReadTx PGSourceConfig
sourceConfig TxET QErr IO (Maybe [PGSourceLockQuery])
fetchLockedQueriesTx
  case Either QErr (Maybe [PGSourceLockQuery])
dbStats of
    Left QErr
err -> Logger Hasura
-> forall a (m :: * -> *).
   (ToEngineLog a Hasura, MonadIO m) =>
   a -> m ()
forall impl.
Logger impl
-> forall a (m :: * -> *).
   (ToEngineLog a impl, MonadIO m) =>
   a -> m ()
unLogger Logger Hasura
logger (PGSourceLockQueryError -> m ()) -> PGSourceLockQueryError -> m ()
forall a b. (a -> b) -> a -> b
$ QErr -> PGSourceLockQueryError
PGSourceLockQueryError QErr
err
    Right (Maybe [PGSourceLockQuery]
val :: (Maybe [PGSourceLockQuery])) ->
      case Maybe [PGSourceLockQuery]
val of
        Maybe [PGSourceLockQuery]
Nothing -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
        Just [] -> () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()
        Just [PGSourceLockQuery]
val' -> IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ Logger Hasura
-> forall a (m :: * -> *).
   (ToEngineLog a Hasura, MonadIO m) =>
   a -> m ()
forall impl.
Logger impl
-> forall a (m :: * -> *).
   (ToEngineLog a impl, MonadIO m) =>
   a -> m ()
unLogger Logger Hasura
logger ([PGSourceLockQuery] -> IO ()) -> [PGSourceLockQuery] -> IO ()
forall a b. (a -> b) -> a -> b
$ [PGSourceLockQuery]
val'
  IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ DiffTime -> IO ()
sleep (DiffTime -> IO ()) -> DiffTime -> IO ()
forall a b. (a -> b) -> a -> b
$ Seconds -> DiffTime
seconds Seconds
5
  where
    -- The blocking query in the below transaction is truncated to the first 20 characters because it may contain
    -- sensitive info.
    fetchLockedQueriesTx :: TxET QErr IO (Maybe [PGSourceLockQuery])
fetchLockedQueriesTx =
      (ViaJSON (Maybe [PGSourceLockQuery]) -> Maybe [PGSourceLockQuery]
forall a. ViaJSON a -> a
PG.getViaJSON (ViaJSON (Maybe [PGSourceLockQuery]) -> Maybe [PGSourceLockQuery])
-> (SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery])))
    -> ViaJSON (Maybe [PGSourceLockQuery]))
-> SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery])))
-> Maybe [PGSourceLockQuery]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Identity (ViaJSON (Maybe [PGSourceLockQuery]))
-> ViaJSON (Maybe [PGSourceLockQuery])
forall a. Identity a -> a
runIdentity (Identity (ViaJSON (Maybe [PGSourceLockQuery]))
 -> ViaJSON (Maybe [PGSourceLockQuery]))
-> (SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery])))
    -> Identity (ViaJSON (Maybe [PGSourceLockQuery])))
-> SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery])))
-> ViaJSON (Maybe [PGSourceLockQuery])
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery])))
-> Identity (ViaJSON (Maybe [PGSourceLockQuery]))
forall a. SingleRow a -> a
PG.getRow)
        (SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery])))
 -> Maybe [PGSourceLockQuery])
-> TxET
     QErr
     IO
     (SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery]))))
-> TxET QErr IO (Maybe [PGSourceLockQuery])
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (PGTxErr -> QErr)
-> Query
-> ()
-> Bool
-> TxET
     QErr
     IO
     (SingleRow (Identity (ViaJSON (Maybe [PGSourceLockQuery]))))
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(json_agg(DISTINCT jsonb_build_object('query', psa.query, 'lock_granted', pl.granted, 'lock_mode', pl.mode, 'transaction_start_time', psa.xact_start, 'query_start_time', psa.query_start, 'wait_event_type', psa.wait_event_type, 'blocking_query', (SUBSTRING(blocking.query, 1, 20) || '...') )), '[]'::json)
         FROM     pg_stat_activity psa
         JOIN     pg_stat_activity blocking ON blocking.pid = ANY(pg_blocking_pids(psa.pid))
         LEFT JOIN pg_locks pl ON psa.pid = pl.pid
         WHERE    psa.query ILIKE '%hdb_catalog%' AND psa.wait_event_type IS NOT NULL
         AND      psa.query ILIKE any (array ['%create%', '%drop%', '%alter%']);
       |]
          ()
          Bool
False

resolveDatabaseMetadata ::
  forall pgKind m.
  ( Backend ('Postgres pgKind),
    ToMetadataFetchQuery pgKind,
    FetchFunctionMetadata pgKind,
    FetchTableMetadata pgKind,
    MonadIO m,
    MonadBaseControl IO m
  ) =>
  SourceMetadata ('Postgres pgKind) ->
  SourceConfig ('Postgres pgKind) ->
  m (Either QErr (DBObjectsIntrospection ('Postgres pgKind)))
resolveDatabaseMetadata :: forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 FetchFunctionMetadata pgKind, FetchTableMetadata pgKind, MonadIO m,
 MonadBaseControl IO m) =>
SourceMetadata ('Postgres pgKind)
-> SourceConfig ('Postgres pgKind)
-> m (Either QErr (DBObjectsIntrospection ('Postgres pgKind)))
resolveDatabaseMetadata SourceMetadata ('Postgres pgKind)
sourceMetadata SourceConfig ('Postgres pgKind)
sourceConfig =
  ExceptT QErr m (DBObjectsIntrospection ('Postgres pgKind))
-> m (Either QErr (DBObjectsIntrospection ('Postgres pgKind)))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT QErr m (DBObjectsIntrospection ('Postgres pgKind))
 -> m (Either QErr (DBObjectsIntrospection ('Postgres pgKind))))
-> ExceptT QErr m (DBObjectsIntrospection ('Postgres pgKind))
-> m (Either QErr (DBObjectsIntrospection ('Postgres pgKind)))
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
PG.ReadOnly Maybe TxIsolation
forall a. Maybe a
Nothing) PGExecFrom
InternalRawQuery) do
    HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
tablesMeta <- HashSet QualifiedTable
-> TxET QErr 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 QualifiedTable
 -> TxET QErr m (DBTablesMetadata ('Postgres pgKind)))
-> HashSet QualifiedTable
-> TxET QErr m (DBTablesMetadata ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ HashMap QualifiedTable (TableMetadata ('Postgres pgKind))
-> HashSet QualifiedTable
forall k a. HashMap k a -> HashSet k
HashMap.keysSet (HashMap QualifiedTable (TableMetadata ('Postgres pgKind))
 -> HashSet QualifiedTable)
-> HashMap QualifiedTable (TableMetadata ('Postgres pgKind))
-> HashSet QualifiedTable
forall a b. (a -> b) -> a -> b
$ InsOrdHashMap QualifiedTable (TableMetadata ('Postgres pgKind))
-> HashMap QualifiedTable (TableMetadata ('Postgres pgKind))
forall k v. InsOrdHashMap k v -> HashMap k v
InsOrdHashMap.toHashMap (InsOrdHashMap QualifiedTable (TableMetadata ('Postgres pgKind))
 -> HashMap QualifiedTable (TableMetadata ('Postgres pgKind)))
-> InsOrdHashMap QualifiedTable (TableMetadata ('Postgres pgKind))
-> HashMap QualifiedTable (TableMetadata ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ SourceMetadata ('Postgres pgKind) -> Tables ('Postgres pgKind)
forall (b :: BackendType). SourceMetadata b -> Tables b
_smTables SourceMetadata ('Postgres pgKind)
sourceMetadata
    let allFunctions :: HashSet QualifiedFunction
allFunctions =
          [QualifiedFunction] -> HashSet QualifiedFunction
forall a. (Eq a, Hashable a) => [a] -> HashSet a
Set.fromList
            ([QualifiedFunction] -> HashSet QualifiedFunction)
-> [QualifiedFunction] -> HashSet QualifiedFunction
forall a b. (a -> b) -> a -> b
$ InsOrdHashMap
  QualifiedFunction (FunctionMetadata ('Postgres pgKind))
-> [QualifiedFunction]
forall k v. InsOrdHashMap k v -> [k]
InsOrdHashMap.keys (SourceMetadata ('Postgres pgKind) -> Functions ('Postgres pgKind)
forall (b :: BackendType). SourceMetadata b -> Functions b
_smFunctions SourceMetadata ('Postgres pgKind)
sourceMetadata) -- Tracked functions
            [QualifiedFunction] -> [QualifiedFunction] -> [QualifiedFunction]
forall a. Semigroup a => a -> a -> a
<> (TableMetadata ('Postgres pgKind) -> [QualifiedFunction])
-> [TableMetadata ('Postgres pgKind)] -> [QualifiedFunction]
forall (t :: * -> *) a b. Foldable t => (a -> [b]) -> t a -> [b]
concatMap TableMetadata ('Postgres pgKind)
-> [FunctionName ('Postgres pgKind)]
TableMetadata ('Postgres pgKind) -> [QualifiedFunction]
getComputedFieldFunctionsMetadata (InsOrdHashMap QualifiedTable (TableMetadata ('Postgres pgKind))
-> [TableMetadata ('Postgres pgKind)]
forall k v. InsOrdHashMap k v -> [v]
InsOrdHashMap.elems (InsOrdHashMap QualifiedTable (TableMetadata ('Postgres pgKind))
 -> [TableMetadata ('Postgres pgKind)])
-> InsOrdHashMap QualifiedTable (TableMetadata ('Postgres pgKind))
-> [TableMetadata ('Postgres pgKind)]
forall a b. (a -> b) -> a -> b
$ SourceMetadata ('Postgres pgKind) -> Tables ('Postgres pgKind)
forall (b :: BackendType). SourceMetadata b -> Tables b
_smTables SourceMetadata ('Postgres pgKind)
sourceMetadata) -- Computed field functions
    HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionsMeta <- forall (pgKind :: PostgresKind) (m :: * -> *).
(FetchFunctionMetadata pgKind, MonadTx m) =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres pgKind))
fetchFunctionMetadata @pgKind HashSet QualifiedFunction
allFunctions
    HashSet PGScalarType
pgScalars <- TxET QErr m (HashSet PGScalarType)
forall (m :: * -> *). MonadTx m => m (HashSet PGScalarType)
fetchPgScalars
    let scalarsMap :: HashMap Name PGScalarType
scalarsMap = [(Name, PGScalarType)] -> HashMap Name PGScalarType
forall k v. (Eq k, Hashable k) => [(k, v)] -> HashMap k v
HashMap.fromList do
          PGScalarType
scalar <- HashSet PGScalarType -> [PGScalarType]
forall a. HashSet a -> [a]
Set.toList HashSet PGScalarType
pgScalars
          Name
name <- forall (t :: * -> *) (f :: * -> *) a.
(Foldable t, Alternative f) =>
t a -> f a
afold @(Either QErr) (Either QErr Name -> [Name]) -> Either QErr Name -> [Name]
forall a b. (a -> b) -> a -> b
$ PGScalarType -> Either QErr Name
forall (m :: * -> *). MonadError QErr m => PGScalarType -> m Name
mkScalarTypeName PGScalarType
scalar
          (Name, PGScalarType) -> [(Name, PGScalarType)]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Name
name, PGScalarType
scalar)
    DBObjectsIntrospection ('Postgres pgKind)
-> TxET QErr m (DBObjectsIntrospection ('Postgres pgKind))
forall a. a -> TxET QErr m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (DBObjectsIntrospection ('Postgres pgKind)
 -> TxET QErr m (DBObjectsIntrospection ('Postgres pgKind)))
-> DBObjectsIntrospection ('Postgres pgKind)
-> TxET QErr m (DBObjectsIntrospection ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ DBTablesMetadata ('Postgres pgKind)
-> DBFunctionsMetadata ('Postgres pgKind)
-> ScalarMap ('Postgres pgKind)
-> LogicalModels ('Postgres pgKind)
-> DBObjectsIntrospection ('Postgres pgKind)
forall (b :: BackendType).
DBTablesMetadata b
-> DBFunctionsMetadata b
-> ScalarMap b
-> LogicalModels b
-> DBObjectsIntrospection b
DBObjectsIntrospection DBTablesMetadata ('Postgres pgKind)
HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
tablesMeta DBFunctionsMetadata ('Postgres pgKind)
HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
functionsMeta (HashMap Name (ScalarType ('Postgres pgKind))
-> ScalarMap ('Postgres pgKind)
forall (b :: BackendType).
HashMap Name (ScalarType b) -> ScalarMap b
ScalarMap HashMap Name (ScalarType ('Postgres pgKind))
HashMap Name PGScalarType
scalarsMap) LogicalModels ('Postgres pgKind)
forall a. Monoid a => a
mempty
  where
    -- A helper function to list all functions underpinning computed fields from a table metadata
    getComputedFieldFunctionsMetadata :: TableMetadata ('Postgres pgKind) -> [FunctionName ('Postgres pgKind)]
    getComputedFieldFunctionsMetadata :: TableMetadata ('Postgres pgKind)
-> [FunctionName ('Postgres pgKind)]
getComputedFieldFunctionsMetadata =
      (ComputedFieldMetadata ('Postgres pgKind) -> QualifiedFunction)
-> [ComputedFieldMetadata ('Postgres pgKind)]
-> [QualifiedFunction]
forall a b. (a -> b) -> [a] -> [b]
map (ComputedFieldDefinition -> QualifiedFunction
_cfdFunction (ComputedFieldDefinition -> QualifiedFunction)
-> (ComputedFieldMetadata ('Postgres pgKind)
    -> ComputedFieldDefinition)
-> ComputedFieldMetadata ('Postgres pgKind)
-> QualifiedFunction
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ComputedFieldMetadata ('Postgres pgKind)
-> ComputedFieldDefinition ('Postgres pgKind)
ComputedFieldMetadata ('Postgres pgKind) -> ComputedFieldDefinition
forall (b :: BackendType).
ComputedFieldMetadata b -> ComputedFieldDefinition b
_cfmDefinition) ([ComputedFieldMetadata ('Postgres pgKind)] -> [QualifiedFunction])
-> (TableMetadata ('Postgres pgKind)
    -> [ComputedFieldMetadata ('Postgres pgKind)])
-> TableMetadata ('Postgres pgKind)
-> [QualifiedFunction]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. InsOrdHashMap
  ComputedFieldName (ComputedFieldMetadata ('Postgres pgKind))
-> [ComputedFieldMetadata ('Postgres pgKind)]
forall k v. InsOrdHashMap k v -> [v]
InsOrdHashMap.elems (InsOrdHashMap
   ComputedFieldName (ComputedFieldMetadata ('Postgres pgKind))
 -> [ComputedFieldMetadata ('Postgres pgKind)])
-> (TableMetadata ('Postgres pgKind)
    -> InsOrdHashMap
         ComputedFieldName (ComputedFieldMetadata ('Postgres pgKind)))
-> TableMetadata ('Postgres pgKind)
-> [ComputedFieldMetadata ('Postgres pgKind)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. TableMetadata ('Postgres pgKind)
-> InsOrdHashMap
     ComputedFieldName (ComputedFieldMetadata ('Postgres pgKind))
forall (b :: BackendType). TableMetadata b -> ComputedFields b
_tmComputedFields

-- | Initialise catalog tables for a source, including those required by the event delivery subsystem.
prepareCatalog ::
  (MonadIO m, MonadBaseControl IO m) =>
  SourceConfig ('Postgres pgKind) ->
  ExceptT QErr m (RecreateEventTriggers, Version.SourceCatalogMigrationState)
-- TODO: SHould we prepare the catalog in Serializable isolation mode like the 'prepareCatalog' of MSSQL
prepareCatalog :: forall (m :: * -> *) (pgKind :: PostgresKind).
(MonadIO m, MonadBaseControl IO m) =>
SourceConfig ('Postgres pgKind)
-> ExceptT
     QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
prepareCatalog SourceConfig ('Postgres pgKind)
sourceConfig = PGExecCtx -> PGExecCtxInfo -> RunTx
_pecRunTx (PGSourceConfig -> PGExecCtx
_pscExecCtx SourceConfig ('Postgres pgKind)
PGSourceConfig
sourceConfig) (PGExecTxType -> PGExecFrom -> PGExecCtxInfo
PGExecCtxInfo (TxAccess -> Maybe TxIsolation -> PGExecTxType
Tx TxAccess
PG.ReadWrite Maybe TxIsolation
forall a. Maybe a
Nothing) PGExecFrom
InternalRawQuery) do
  Bool
hdbCatalogExist <- SchemaName -> TxET QErr m Bool
forall (m :: * -> *). MonadTx m => SchemaName -> m Bool
doesSchemaExist SchemaName
"hdb_catalog"
  Bool
eventLogTableExist <- SchemaName -> TableName -> TxET QErr m Bool
forall (m :: * -> *).
MonadTx m =>
SchemaName -> TableName -> m Bool
doesTableExist SchemaName
"hdb_catalog" TableName
"event_log"
  Bool
sourceVersionTableExist <- SchemaName -> TableName -> TxET QErr m Bool
forall (m :: * -> *).
MonadTx m =>
SchemaName -> TableName -> m Bool
doesTableExist SchemaName
"hdb_catalog" TableName
"hdb_source_catalog_version"
  if
    -- Fresh database
    | Bool -> Bool
not Bool
hdbCatalogExist -> TxE QErr (RecreateEventTriggers, SourceCatalogMigrationState)
-> TxET QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. TxE QErr a -> TxET QErr m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx do
        (PGTxErr -> QErr) -> Query -> () -> Bool -> TxET QErr IO ()
forall (m :: * -> *) r e.
(MonadIO m, ToPrepArgs r) =>
(PGTxErr -> e) -> Query -> r -> Bool -> TxET e m ()
PG.unitQE PGTxErr -> QErr
defaultTxErrorHandler Query
"CREATE SCHEMA hdb_catalog" () Bool
False
        ExtensionsSchema -> TxET QErr IO ()
forall (m :: * -> *). MonadTx m => ExtensionsSchema -> m ()
enablePgcryptoExtension (ExtensionsSchema -> TxET QErr IO ())
-> ExtensionsSchema -> TxET QErr IO ()
forall a b. (a -> b) -> a -> b
$ PGSourceConfig -> ExtensionsSchema
_pscExtensionsSchema SourceConfig ('Postgres pgKind)
PGSourceConfig
sourceConfig
        TxET QErr IO ()
initPgSourceCatalog
        (RecreateEventTriggers, SourceCatalogMigrationState)
-> TxE QErr (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. a -> TxET QErr IO a
forall (m :: * -> *) a. Monad m => a -> m a
return (RecreateEventTriggers
RETDoNothing, Int -> SourceCatalogMigrationState
Version.SCMSInitialized (Int -> SourceCatalogMigrationState)
-> Int -> SourceCatalogMigrationState
forall a b. (a -> b) -> a -> b
$ SourceCatalogVersion ('Postgres Any) -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion ('Postgres Any)
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
latestSourceCatalogVersion)
    -- Only 'hdb_catalog' schema defined
    | Bool -> Bool
not (Bool
sourceVersionTableExist Bool -> Bool -> Bool
|| Bool
eventLogTableExist) -> do
        TxET QErr IO () -> TxET QErr m ()
forall a. TxE QErr a -> TxET QErr m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx TxET QErr IO ()
initPgSourceCatalog
        (RecreateEventTriggers, SourceCatalogMigrationState)
-> TxET QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. a -> TxET QErr m a
forall (m :: * -> *) a. Monad m => a -> m a
return (RecreateEventTriggers
RETDoNothing, Int -> SourceCatalogMigrationState
Version.SCMSInitialized (Int -> SourceCatalogMigrationState)
-> Int -> SourceCatalogMigrationState
forall a b. (a -> b) -> a -> b
$ SourceCatalogVersion ('Postgres Any) -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion ('Postgres Any)
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
latestSourceCatalogVersion)
    -- Source is initialised by pre multisource support servers
    | Bool -> Bool
not Bool
sourceVersionTableExist Bool -> Bool -> Bool
&& Bool
eventLogTableExist -> do
        -- Update the Source Catalog to v43 to include the new migration
        -- changes. Skipping this step will result in errors.
        MetadataCatalogVersion
currMetadataCatalogVersion <- TxE QErr MetadataCatalogVersion
-> TxET QErr m MetadataCatalogVersion
forall a. TxE QErr a -> TxET QErr m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx TxE QErr MetadataCatalogVersion
getCatalogVersion
        -- we migrate to the 43 version, which is the migration where
        -- metadata separation is introduced
        MetadataCatalogVersion -> TxET QErr m ()
forall {m :: * -> *}.
(MonadIO m, MonadTx m) =>
MetadataCatalogVersion -> m ()
migrateTo43MetadataCatalog MetadataCatalogVersion
currMetadataCatalogVersion
        TxET QErr IO () -> TxET QErr m ()
forall a. TxE QErr a -> TxET QErr m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx TxET QErr IO ()
createVersionTable
        -- Migrate the catalog from initial version i.e '0'
        SourceCatalogVersion ('Postgres Any)
-> TxET QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
forall (m :: * -> *) (pgKind :: PostgresKind).
MonadTx m =>
SourceCatalogVersion pgKind
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalogFrom SourceCatalogVersion ('Postgres Any)
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
initialSourceCatalogVersion
    | Bool
otherwise -> TxET QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
forall (m :: * -> *).
MonadTx m =>
m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalog
  where
    initPgSourceCatalog :: TxET QErr IO ()
initPgSourceCatalog = do
      () <- (PGTxErr -> QErr) -> Query -> TxET QErr IO ()
forall (m :: * -> *) a e.
(MonadIO m, FromRes a) =>
(PGTxErr -> e) -> Query -> TxET e m a
PG.multiQE PGTxErr -> QErr
defaultTxErrorHandler $(makeRelativeToProject "src-rsr/init_pg_source.sql" >>= PG.sqlFromFile)
      TxET QErr IO ()
forall (m :: * -> *). MonadTx m => m ()
setSourceCatalogVersion

    createVersionTable :: TxET QErr IO ()
createVersionTable = do
      () <-
        (PGTxErr -> QErr) -> Query -> TxET QErr IO ()
forall (m :: * -> *) a e.
(MonadIO m, FromRes a) =>
(PGTxErr -> e) -> Query -> TxET e m a
PG.multiQE
          PGTxErr -> QErr
defaultTxErrorHandler
          [PG.sql|
           CREATE TABLE hdb_catalog.hdb_source_catalog_version(
             version TEXT NOT NULL,
             upgraded_on TIMESTAMPTZ NOT NULL
           );

           CREATE UNIQUE INDEX hdb_source_catalog_version_one_row
           ON hdb_catalog.hdb_source_catalog_version((version IS NOT NULL));
        |]
      () -> TxET QErr IO ()
forall a. a -> TxET QErr IO a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

    migrateTo43MetadataCatalog :: MetadataCatalogVersion -> m ()
migrateTo43MetadataCatalog MetadataCatalogVersion
prevVersion = do
      let neededMigrations :: [(MetadataCatalogVersion, TxET QErr IO ())]
neededMigrations = ((MetadataCatalogVersion, TxET QErr IO ()) -> Bool)
-> [(MetadataCatalogVersion, TxET QErr IO ())]
-> [(MetadataCatalogVersion, TxET QErr IO ())]
forall a. (a -> Bool) -> [a] -> [a]
dropWhile ((MetadataCatalogVersion -> MetadataCatalogVersion -> Bool
forall a. Ord a => a -> a -> Bool
< MetadataCatalogVersion
prevVersion) (MetadataCatalogVersion -> Bool)
-> ((MetadataCatalogVersion, TxET QErr IO ())
    -> MetadataCatalogVersion)
-> (MetadataCatalogVersion, TxET QErr IO ())
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (MetadataCatalogVersion, TxET QErr IO ()) -> MetadataCatalogVersion
forall a b. (a, b) -> a
fst) [(MetadataCatalogVersion, TxET QErr IO ())]
upMigrationsUntil43
      case [(MetadataCatalogVersion, TxET QErr IO ())]
-> Maybe (NonEmpty (MetadataCatalogVersion, TxET QErr IO ()))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [(MetadataCatalogVersion, TxET QErr IO ())]
neededMigrations of
        Just NonEmpty (MetadataCatalogVersion, TxET QErr IO ())
nonEmptyNeededMigrations -> do
          -- Migrations aren't empty. We need to update the catalog version after migrations
          UTCTime
migrationTime <- IO UTCTime -> m UTCTime
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO IO UTCTime
getCurrentTime
          TxET QErr IO () -> m ()
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx (TxET QErr IO () -> m ()) -> TxET QErr IO () -> m ()
forall a b. (a -> b) -> a -> b
$ ((MetadataCatalogVersion, TxET QErr IO ()) -> TxET QErr IO ())
-> NonEmpty (MetadataCatalogVersion, TxET QErr IO ())
-> TxET QErr IO ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
(a -> f b) -> t a -> f ()
traverse_ (MetadataCatalogVersion, TxET QErr IO ()) -> TxET QErr IO ()
forall a b. (a, b) -> b
snd NonEmpty (MetadataCatalogVersion, TxET QErr IO ())
nonEmptyNeededMigrations
          Text -> UTCTime -> m ()
forall (m :: * -> *). MonadTx m => Text -> UTCTime -> m ()
setCatalogVersion Text
"43" UTCTime
migrationTime
        Maybe (NonEmpty (MetadataCatalogVersion, TxET QErr IO ()))
Nothing ->
          -- No migrations exists, implies the database is migrated to latest metadata catalog version
          () -> m ()
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ()

-- NOTE (rakesh):
-- Down migrations for postgres sources is not supported in this PR. We need an
-- exhaustive discussion to make a call as I think, as of now, it is not
-- trivial. For metadata catalog migrations, we have a separate downgrade
-- command in the graphql-engine exe.
--
-- I can think of two ways:
--
--  - Just like downgrade, we need to have a new command path for downgrading
--  pg sources (command design should support other backends too,
--  graphql-engine source-downgrade postgres --to-catalog-version 1 --
--  downgrade all available pg sources to 1)
--  - Have an online documentation with necessary SQLs to help users to
--  downgrade pg sources themselves. Improve error message by referring the URL
--  to the documentation.

migrateSourceCatalog :: (MonadTx m) => m (RecreateEventTriggers, Version.SourceCatalogMigrationState)
migrateSourceCatalog :: forall (m :: * -> *).
MonadTx m =>
m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalog =
  m (SourceCatalogVersion ('Postgres Any))
forall (m :: * -> *) (postgres :: PostgresKind).
MonadTx m =>
m (SourceCatalogVersion postgres)
getSourceCatalogVersion m (SourceCatalogVersion ('Postgres Any))
-> (SourceCatalogVersion ('Postgres Any)
    -> m (RecreateEventTriggers, SourceCatalogMigrationState))
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a b. m a -> (a -> m b) -> m b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= SourceCatalogVersion ('Postgres Any)
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall (m :: * -> *) (pgKind :: PostgresKind).
MonadTx m =>
SourceCatalogVersion pgKind
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalogFrom

-- | `migrateSourceCatalogFrom` migrates the catalog from a lower to a higher version.
--    When there are any changes in the source catalog, then re-create the existing event
--    triggers in the metadata. This is done so that the event triggers be compatible with the
--    changes introduced in the newly added source catalog migrations. When the source is already
--    in the latest catalog version, we do nothing because nothing has changed w.r.t the source catalog
--    so recreating the event triggers will only be extraneous.
migrateSourceCatalogFrom ::
  (MonadTx m) =>
  SourceCatalogVersion pgKind ->
  m (RecreateEventTriggers, Version.SourceCatalogMigrationState)
migrateSourceCatalogFrom :: forall (m :: * -> *) (pgKind :: PostgresKind).
MonadTx m =>
SourceCatalogVersion pgKind
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalogFrom SourceCatalogVersion pgKind
prevVersion
  | SourceCatalogVersion pgKind
prevVersion SourceCatalogVersion pgKind -> SourceCatalogVersion pgKind -> Bool
forall a. Eq a => a -> a -> Bool
== SourceCatalogVersion pgKind
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
latestSourceCatalogVersion = (RecreateEventTriggers, SourceCatalogMigrationState)
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (RecreateEventTriggers
RETDoNothing, Int -> SourceCatalogMigrationState
Version.SCMSNothingToDo (Int -> SourceCatalogMigrationState)
-> Int -> SourceCatalogMigrationState
forall a b. (a -> b) -> a -> b
$ SourceCatalogVersion ('Postgres Any) -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion ('Postgres Any)
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
latestSourceCatalogVersion)
  | [] <- [(SourceCatalogVersion pgKind, TxET QErr IO ())]
neededMigrations =
      Code
-> Text -> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall (m :: * -> *) a. QErrM m => Code -> Text -> m a
throw400 Code
NotSupported
        (Text -> m (RecreateEventTriggers, SourceCatalogMigrationState))
-> Text -> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a b. (a -> b) -> a -> b
$ Text
"Expected source catalog version <= "
        Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> SourceCatalogVersion ('Postgres Any) -> Text
forall a. Show a => a -> Text
tshow SourceCatalogVersion ('Postgres Any)
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
latestSourceCatalogVersion
        Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
", but the current version is "
        Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> SourceCatalogVersion pgKind -> Text
forall a. Show a => a -> Text
tshow SourceCatalogVersion pgKind
prevVersion
  | Bool
otherwise = do
      TxET QErr IO () -> m ()
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx (TxET QErr IO () -> m ()) -> TxET QErr IO () -> m ()
forall a b. (a -> b) -> a -> b
$ ((SourceCatalogVersion pgKind, TxET QErr IO ()) -> TxET QErr IO ())
-> [(SourceCatalogVersion pgKind, TxET QErr IO ())]
-> TxET QErr IO ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
(a -> f b) -> t a -> f ()
traverse_ (SourceCatalogVersion pgKind, TxET QErr IO ()) -> TxET QErr IO ()
forall a b. (a, b) -> b
snd [(SourceCatalogVersion pgKind, TxET QErr IO ())]
neededMigrations
      m ()
forall (m :: * -> *). MonadTx m => m ()
setSourceCatalogVersion
      (RecreateEventTriggers, SourceCatalogMigrationState)
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( RecreateEventTriggers
RETRecreate,
          Int -> Int -> SourceCatalogMigrationState
Version.SCMSMigratedTo
            (SourceCatalogVersion pgKind -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion pgKind
prevVersion)
            (SourceCatalogVersion ('Postgres Any) -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion ('Postgres Any)
forall (pgKind :: PostgresKind). SourceCatalogVersion pgKind
latestSourceCatalogVersion)
        )
  where
    neededMigrations :: [(SourceCatalogVersion pgKind, TxET QErr IO ())]
neededMigrations =
      ((SourceCatalogVersion pgKind, TxET QErr IO ()) -> Bool)
-> [(SourceCatalogVersion pgKind, TxET QErr IO ())]
-> [(SourceCatalogVersion pgKind, TxET QErr IO ())]
forall a. (a -> Bool) -> [a] -> [a]
dropWhile ((SourceCatalogVersion pgKind -> SourceCatalogVersion pgKind -> Bool
forall a. Eq a => a -> a -> Bool
/= SourceCatalogVersion pgKind
prevVersion) (SourceCatalogVersion pgKind -> Bool)
-> ((SourceCatalogVersion pgKind, TxET QErr IO ())
    -> SourceCatalogVersion pgKind)
-> (SourceCatalogVersion pgKind, TxET QErr IO ())
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SourceCatalogVersion pgKind, TxET QErr IO ())
-> SourceCatalogVersion pgKind
forall a b. (a, b) -> a
fst) [(SourceCatalogVersion pgKind, TxET QErr IO ())]
forall (pgKind :: PostgresKind).
[(SourceCatalogVersion pgKind, TxET QErr IO ())]
sourceMigrations

sourceMigrations :: [(SourceCatalogVersion pgKind, PG.TxE QErr ())]
sourceMigrations :: forall (pgKind :: PostgresKind).
[(SourceCatalogVersion pgKind, TxET QErr IO ())]
sourceMigrations =
  $( let migrationFromFile from =
           let to = succ from
               path = "src-rsr/pg_source_migrations/" <> show from <> "_to_" <> show to <> ".sql"
            in [|PG.multiQE defaultTxErrorHandler $(makeRelativeToProject path >>= PG.sqlFromFile)|]

         migrationsFromFile = map $ \from ->
           [|($(TH.lift from), $(migrationFromFile from))|]
      in TH.listE $ migrationsFromFile previousSourceCatalogVersions
   )

-- Upgrade the hdb_catalog schema to v43 (Metadata catalog)
upMigrationsUntil43 :: [(Version.MetadataCatalogVersion, PG.TxE QErr ())]
upMigrationsUntil43 :: [(MetadataCatalogVersion, TxET QErr IO ())]
upMigrationsUntil43 =
  $( let migrationFromFile from to =
           let path = "src-rsr/migrations/" <> from <> "_to_" <> to <> ".sql"
            in [|PG.multiQE defaultTxErrorHandler $(makeRelativeToProject path >>= PG.sqlFromFile)|]

         migrationsFromFile = map $ \(to :: Int) ->
           let from = pred to
            in [|
                 ( $(TH.lift (Version.MetadataCatalogVersion from)),
                   $(migrationFromFile (show from) (show to))
                 )
                 |]
      in TH.listE
           -- version 0.8 is the only non-integral catalog version
           -- The 41st migration which included only source catalog migration
           -- was introduced before metadata separation changes were introduced
           -- in the graphql-engine. Now the earlier 41st migration has been
           -- moved to source catalog migrations and the 41st up migration is removed
           -- entirely.
           $ [|(Version.MetadataCatalogVersion08, $(migrationFromFile "08" "1"))|]
           : migrationsFromFile [2 .. 3]
             ++ [|(Version.MetadataCatalogVersion 3, from3To4)|]
           : migrationsFromFile [5 .. 40]
             ++ migrationsFromFile [42 .. 43]
   )

-- | We differentiate for CockroachDB and other PG implementations
-- as our CockroachDB table fetching SQL does not require table information,
-- and fails if it receives unused prepared arguments
-- this distinction should no longer be necessary if this issue is resolved:
-- https://github.com/cockroachdb/cockroach/issues/86375
class FetchTableMetadata (pgKind :: PostgresKind) where
  fetchTableMetadata ::
    forall m.
    ( Backend ('Postgres pgKind),
      ToMetadataFetchQuery pgKind,
      MonadTx m
    ) =>
    Set.HashSet QualifiedTable ->
    m (DBTablesMetadata ('Postgres pgKind))

instance FetchTableMetadata 'Vanilla where
  fetchTableMetadata :: forall (m :: * -> *).
(Backend ('Postgres 'Vanilla), ToMetadataFetchQuery 'Vanilla,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres 'Vanilla))
fetchTableMetadata = HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres 'Vanilla))
forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
pgFetchTableMetadata

instance FetchTableMetadata 'Citus where
  fetchTableMetadata :: forall (m :: * -> *).
(Backend ('Postgres 'Citus), ToMetadataFetchQuery 'Citus,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres 'Citus))
fetchTableMetadata = HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres 'Citus))
forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
pgFetchTableMetadata

instance FetchTableMetadata 'Cockroach where
  fetchTableMetadata :: forall (m :: * -> *).
(Backend ('Postgres 'Cockroach), ToMetadataFetchQuery 'Cockroach,
 MonadTx m) =>
HashSet QualifiedTable
-> m (DBTablesMetadata ('Postgres 'Cockroach))
fetchTableMetadata = HashSet QualifiedTable
-> m (DBTablesMetadata ('Postgres 'Cockroach))
forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
cockroachFetchTableMetadata

-- | Fetch Postgres metadata of all user tables
pgFetchTableMetadata ::
  forall pgKind m.
  (Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind, MonadTx m) =>
  Set.HashSet QualifiedTable ->
  m (DBTablesMetadata ('Postgres pgKind))
pgFetchTableMetadata :: forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
pgFetchTableMetadata HashSet QualifiedTable
tables = do
  [(SchemaName, TableName,
  ViaJSON (DBTableMetadata ('Postgres pgKind)))]
results <-
    TxE
  QErr
  [(SchemaName, TableName,
    ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> m [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx
      (TxE
   QErr
   [(SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))]
 -> m [(SchemaName, TableName,
        ViaJSON (DBTableMetadata ('Postgres pgKind)))])
-> TxE
     QErr
     [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> m [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
forall a b. (a -> b) -> a -> b
$ (PGTxErr -> QErr)
-> Query
-> [ViaJSON [QualifiedTable]]
-> Bool
-> TxE
     QErr
     [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
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
        (forall (pgKind :: PostgresKind).
ToMetadataFetchQuery pgKind =>
Query
tableMetadata @pgKind)
        [[QualifiedTable] -> ViaJSON [QualifiedTable]
forall a. a -> ViaJSON a
PG.ViaJSON ([QualifiedTable] -> ViaJSON [QualifiedTable])
-> [QualifiedTable] -> ViaJSON [QualifiedTable]
forall a b. (a -> b) -> a -> b
$ [QualifiedTable] -> [QualifiedTable]
forall a. Ord a => [a] -> [a]
LE.uniques ([QualifiedTable] -> [QualifiedTable])
-> [QualifiedTable] -> [QualifiedTable]
forall a b. (a -> b) -> a -> b
$ HashSet QualifiedTable -> [QualifiedTable]
forall a. HashSet a -> [a]
Set.toList HashSet QualifiedTable
tables]
        Bool
True
  HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
-> m (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
 -> m (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))))
-> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
-> m (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind)))
forall a b. (a -> b) -> a -> b
$ [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
-> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
forall k v. (Eq k, Hashable k) => [(k, v)] -> HashMap k v
HashMap.fromList
    ([(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
 -> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind)))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
-> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ (((SchemaName, TableName,
   ViaJSON (DBTableMetadata ('Postgres pgKind)))
  -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
 -> [(SchemaName, TableName,
      ViaJSON (DBTableMetadata ('Postgres pgKind)))]
 -> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))])
-> [(SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> ((SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))
    -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall a b c. (a -> b -> c) -> b -> a -> c
flip ((SchemaName, TableName,
  ViaJSON (DBTableMetadata ('Postgres pgKind)))
 -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
-> [(SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall a b. (a -> b) -> [a] -> [b]
map [(SchemaName, TableName,
  ViaJSON (DBTableMetadata ('Postgres pgKind)))]
results
    (((SchemaName, TableName,
   ViaJSON (DBTableMetadata ('Postgres pgKind)))
  -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
 -> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))])
-> ((SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))
    -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall a b. (a -> b) -> a -> b
$ \(SchemaName
schema, TableName
table, PG.ViaJSON DBTableMetadata ('Postgres pgKind)
info) -> (SchemaName -> TableName -> QualifiedTable
forall a. SchemaName -> a -> QualifiedObject a
QualifiedObject SchemaName
schema TableName
table, DBTableMetadata ('Postgres pgKind)
info)

-- | Fetch Cockroach metadata of all user tables
cockroachFetchTableMetadata ::
  forall pgKind m.
  (Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind, MonadTx m) =>
  Set.HashSet QualifiedTable ->
  m (DBTablesMetadata ('Postgres pgKind))
cockroachFetchTableMetadata :: forall (pgKind :: PostgresKind) (m :: * -> *).
(Backend ('Postgres pgKind), ToMetadataFetchQuery pgKind,
 MonadTx m) =>
HashSet QualifiedTable -> m (DBTablesMetadata ('Postgres pgKind))
cockroachFetchTableMetadata HashSet QualifiedTable
_tables = do
  [(SchemaName, TableName,
  ViaJSON (DBTableMetadata ('Postgres pgKind)))]
results <-
    TxE
  QErr
  [(SchemaName, TableName,
    ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> m [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx
      (TxE
   QErr
   [(SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))]
 -> m [(SchemaName, TableName,
        ViaJSON (DBTableMetadata ('Postgres pgKind)))])
-> TxE
     QErr
     [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> m [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
forall a b. (a -> b) -> a -> b
$ (PGTxErr -> QErr)
-> Query
-> [PrepArg]
-> Bool
-> TxE
     QErr
     [(SchemaName, TableName,
       ViaJSON (DBTableMetadata ('Postgres pgKind)))]
forall (m :: * -> *) a e.
(MonadIO m, FromRes a) =>
(PGTxErr -> e) -> Query -> [PrepArg] -> Bool -> TxET e m a
PG.rawQE
        PGTxErr -> QErr
defaultTxErrorHandler
        (forall (pgKind :: PostgresKind).
ToMetadataFetchQuery pgKind =>
Query
tableMetadata @pgKind)
        []
        Bool
True
  HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
-> m (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
 -> m (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))))
-> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
-> m (HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind)))
forall a b. (a -> b) -> a -> b
$ [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
-> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
forall k v. (Eq k, Hashable k) => [(k, v)] -> HashMap k v
HashMap.fromList
    ([(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
 -> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind)))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
-> HashMap QualifiedTable (DBTableMetadata ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ (((SchemaName, TableName,
   ViaJSON (DBTableMetadata ('Postgres pgKind)))
  -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
 -> [(SchemaName, TableName,
      ViaJSON (DBTableMetadata ('Postgres pgKind)))]
 -> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))])
-> [(SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> ((SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))
    -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall a b c. (a -> b -> c) -> b -> a -> c
flip ((SchemaName, TableName,
  ViaJSON (DBTableMetadata ('Postgres pgKind)))
 -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
-> [(SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))]
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall a b. (a -> b) -> [a] -> [b]
map [(SchemaName, TableName,
  ViaJSON (DBTableMetadata ('Postgres pgKind)))]
results
    (((SchemaName, TableName,
   ViaJSON (DBTableMetadata ('Postgres pgKind)))
  -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
 -> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))])
-> ((SchemaName, TableName,
     ViaJSON (DBTableMetadata ('Postgres pgKind)))
    -> (QualifiedTable, DBTableMetadata ('Postgres pgKind)))
-> [(QualifiedTable, DBTableMetadata ('Postgres pgKind))]
forall a b. (a -> b) -> a -> b
$ \(SchemaName
schema, TableName
table, PG.ViaJSON DBTableMetadata ('Postgres pgKind)
info) -> (SchemaName -> TableName -> QualifiedTable
forall a. SchemaName -> a -> QualifiedObject a
QualifiedObject SchemaName
schema TableName
table, DBTableMetadata ('Postgres pgKind)
info)

class FetchFunctionMetadata (pgKind :: PostgresKind) where
  fetchFunctionMetadata ::
    (MonadTx m) =>
    Set.HashSet QualifiedFunction ->
    m (DBFunctionsMetadata ('Postgres pgKind))

instance FetchFunctionMetadata 'Vanilla where
  fetchFunctionMetadata :: forall (m :: * -> *).
MonadTx m =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres 'Vanilla))
fetchFunctionMetadata = HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres 'Vanilla))
forall (m :: * -> *) (pgKind :: PostgresKind).
MonadTx m =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres pgKind))
pgFetchFunctionMetadata

instance FetchFunctionMetadata 'Citus where
  fetchFunctionMetadata :: forall (m :: * -> *).
MonadTx m =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres 'Citus))
fetchFunctionMetadata = HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres 'Citus))
forall (m :: * -> *) (pgKind :: PostgresKind).
MonadTx m =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres pgKind))
pgFetchFunctionMetadata

instance FetchFunctionMetadata 'Cockroach where
  fetchFunctionMetadata :: forall (m :: * -> *).
MonadTx m =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres 'Cockroach))
fetchFunctionMetadata HashSet QualifiedFunction
_ = HashMap
  QualifiedFunction (FunctionOverloads ('Postgres 'Cockroach))
-> m (HashMap
        QualifiedFunction (FunctionOverloads ('Postgres 'Cockroach)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure HashMap
  QualifiedFunction (FunctionOverloads ('Postgres 'Cockroach))
forall a. Monoid a => a
mempty

-- | Fetch Postgres metadata for all user functions
pgFetchFunctionMetadata :: (MonadTx m) => Set.HashSet QualifiedFunction -> m (DBFunctionsMetadata ('Postgres pgKind))
pgFetchFunctionMetadata :: forall (m :: * -> *) (pgKind :: PostgresKind).
MonadTx m =>
HashSet QualifiedFunction
-> m (DBFunctionsMetadata ('Postgres pgKind))
pgFetchFunctionMetadata HashSet QualifiedFunction
functions = do
  [(SchemaName, FunctionName,
  ViaJSON (FunctionOverloads ('Postgres pgKind)))]
results <-
    TxE
  QErr
  [(SchemaName, FunctionName,
    ViaJSON (FunctionOverloads ('Postgres pgKind)))]
-> m [(SchemaName, FunctionName,
       ViaJSON (FunctionOverloads ('Postgres pgKind)))]
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx
      (TxE
   QErr
   [(SchemaName, FunctionName,
     ViaJSON (FunctionOverloads ('Postgres pgKind)))]
 -> m [(SchemaName, FunctionName,
        ViaJSON (FunctionOverloads ('Postgres pgKind)))])
-> TxE
     QErr
     [(SchemaName, FunctionName,
       ViaJSON (FunctionOverloads ('Postgres pgKind)))]
-> m [(SchemaName, FunctionName,
       ViaJSON (FunctionOverloads ('Postgres pgKind)))]
forall a b. (a -> b) -> a -> b
$ (PGTxErr -> QErr)
-> Query
-> [ViaJSON (HashSet QualifiedFunction)]
-> Bool
-> TxE
     QErr
     [(SchemaName, FunctionName,
       ViaJSON (FunctionOverloads ('Postgres pgKind)))]
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
        $(makeRelativeToProject "src-rsr/pg_function_metadata.sql" >>= PG.sqlFromFile)
        [HashSet QualifiedFunction -> ViaJSON (HashSet QualifiedFunction)
forall a. a -> ViaJSON a
PG.ViaJSON HashSet QualifiedFunction
functions]
        Bool
True
  HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
-> m (HashMap
        QualifiedFunction (FunctionOverloads ('Postgres pgKind)))
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    (HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
 -> m (HashMap
         QualifiedFunction (FunctionOverloads ('Postgres pgKind))))
-> HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
-> m (HashMap
        QualifiedFunction (FunctionOverloads ('Postgres pgKind)))
forall a b. (a -> b) -> a -> b
$ [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))]
-> HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
forall k v. (Eq k, Hashable k) => [(k, v)] -> HashMap k v
HashMap.fromList
    ([(QualifiedFunction, FunctionOverloads ('Postgres pgKind))]
 -> HashMap
      QualifiedFunction (FunctionOverloads ('Postgres pgKind)))
-> [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))]
-> HashMap QualifiedFunction (FunctionOverloads ('Postgres pgKind))
forall a b. (a -> b) -> a -> b
$ (((SchemaName, FunctionName,
   ViaJSON (FunctionOverloads ('Postgres pgKind)))
  -> (QualifiedFunction, FunctionOverloads ('Postgres pgKind)))
 -> [(SchemaName, FunctionName,
      ViaJSON (FunctionOverloads ('Postgres pgKind)))]
 -> [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))])
-> [(SchemaName, FunctionName,
     ViaJSON (FunctionOverloads ('Postgres pgKind)))]
-> ((SchemaName, FunctionName,
     ViaJSON (FunctionOverloads ('Postgres pgKind)))
    -> (QualifiedFunction, FunctionOverloads ('Postgres pgKind)))
-> [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))]
forall a b c. (a -> b -> c) -> b -> a -> c
flip ((SchemaName, FunctionName,
  ViaJSON (FunctionOverloads ('Postgres pgKind)))
 -> (QualifiedFunction, FunctionOverloads ('Postgres pgKind)))
-> [(SchemaName, FunctionName,
     ViaJSON (FunctionOverloads ('Postgres pgKind)))]
-> [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))]
forall a b. (a -> b) -> [a] -> [b]
map [(SchemaName, FunctionName,
  ViaJSON (FunctionOverloads ('Postgres pgKind)))]
results
    (((SchemaName, FunctionName,
   ViaJSON (FunctionOverloads ('Postgres pgKind)))
  -> (QualifiedFunction, FunctionOverloads ('Postgres pgKind)))
 -> [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))])
-> ((SchemaName, FunctionName,
     ViaJSON (FunctionOverloads ('Postgres pgKind)))
    -> (QualifiedFunction, FunctionOverloads ('Postgres pgKind)))
-> [(QualifiedFunction, FunctionOverloads ('Postgres pgKind))]
forall a b. (a -> b) -> a -> b
$ \(SchemaName
schema, FunctionName
table, PG.ViaJSON FunctionOverloads ('Postgres pgKind)
infos) -> (SchemaName -> FunctionName -> QualifiedFunction
forall a. SchemaName -> a -> QualifiedObject a
QualifiedObject SchemaName
schema FunctionName
table, FunctionOverloads ('Postgres pgKind)
infos)

-- | Fetch all scalar types from Postgres
fetchPgScalars :: (MonadTx m) => m (HashSet PGScalarType)
fetchPgScalars :: forall (m :: * -> *). MonadTx m => m (HashSet PGScalarType)
fetchPgScalars =
  TxE QErr (HashSet PGScalarType) -> m (HashSet PGScalarType)
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx
    (TxE QErr (HashSet PGScalarType) -> m (HashSet PGScalarType))
-> TxE QErr (HashSet PGScalarType) -> m (HashSet PGScalarType)
forall a b. (a -> b) -> a -> b
$ ViaJSON (HashSet PGScalarType) -> HashSet PGScalarType
forall a. ViaJSON a -> a
PG.getViaJSON
    (ViaJSON (HashSet PGScalarType) -> HashSet PGScalarType)
-> (SingleRow (Identity (ViaJSON (HashSet PGScalarType)))
    -> ViaJSON (HashSet PGScalarType))
-> SingleRow (Identity (ViaJSON (HashSet PGScalarType)))
-> HashSet PGScalarType
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Identity (ViaJSON (HashSet PGScalarType))
-> ViaJSON (HashSet PGScalarType)
forall a. Identity a -> a
runIdentity
    (Identity (ViaJSON (HashSet PGScalarType))
 -> ViaJSON (HashSet PGScalarType))
-> (SingleRow (Identity (ViaJSON (HashSet PGScalarType)))
    -> Identity (ViaJSON (HashSet PGScalarType)))
-> SingleRow (Identity (ViaJSON (HashSet PGScalarType)))
-> ViaJSON (HashSet PGScalarType)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. SingleRow (Identity (ViaJSON (HashSet PGScalarType)))
-> Identity (ViaJSON (HashSet PGScalarType))
forall a. SingleRow a -> a
PG.getRow
    (SingleRow (Identity (ViaJSON (HashSet PGScalarType)))
 -> HashSet PGScalarType)
-> TxET
     QErr IO (SingleRow (Identity (ViaJSON (HashSet PGScalarType))))
-> TxE QErr (HashSet PGScalarType)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (PGTxErr -> QErr)
-> Query
-> ()
-> Bool
-> TxET
     QErr IO (SingleRow (Identity (ViaJSON (HashSet PGScalarType))))
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(json_agg(typname), '[]')
    FROM pg_catalog.pg_type where typtype = 'b'
   |]
      ()
      Bool
True

-- | Clean source database after dropping in metadata
postDropSourceHook ::
  (MonadIO m, MonadError QErr m, MonadBaseControl IO m) =>
  SourceConfig ('Postgres pgKind) ->
  TableEventTriggers ('Postgres pgKind) ->
  m ()
postDropSourceHook :: forall (m :: * -> *) (pgKind :: PostgresKind).
(MonadIO m, MonadError QErr m, MonadBaseControl IO m) =>
SourceConfig ('Postgres pgKind)
-> TableEventTriggers ('Postgres pgKind) -> m ()
postDropSourceHook SourceConfig ('Postgres pgKind)
sourceConfig TableEventTriggers ('Postgres pgKind)
tableTriggersMap = do
  -- Clean traces of Hasura in source database
  --
  -- There are three type of database we have to consider here, which we
  -- refer to as types 1, 2, and 3 below:
  --   1. default postgres source (no separate metadata database)
  --   In this case, we want to only drop source-related tables ("event_log",
  --   "hdb_source_catalog_version", etc), leaving the rest of the schema
  --   intact.
  --
  --   2. dedicated metadata database
  --   Ideally a dedicated metadata database won't have any source related
  --   tables. But if it does, then drop only source-related tables, leaving the
  --   rest of schema intact.
  --
  --   3. non-default postgres source (necessarily without metadata tables)
  --   In this case, we want to drop the entire "hdb_catalog" schema.
  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 SourceConfig ('Postgres pgKind)
PGSourceConfig
sourceConfig PGExecFrom
InternalRawQuery
    (TxET QErr m () -> m (Either QErr ()))
-> TxET QErr m () -> m (Either QErr ())
forall a b. (a -> b) -> a -> b
$ do
      Bool
hdbMetadataTableExist <- SchemaName -> TableName -> TxET QErr m Bool
forall (m :: * -> *).
MonadTx m =>
SchemaName -> TableName -> m Bool
doesTableExist SchemaName
"hdb_catalog" TableName
"hdb_metadata"
      if
        -- If "hdb_metadata" exists, we have one of two possible cases:
        --   * this is a metadata database (type 2)
        --   * this is a default database (type 1)
        --
        -- Both of the possible cases might have source-related tables. And in
        -- both the cases we only want to drop the source-related tables
        -- leaving rest of the schema intact.
        --
        -- To adhere to the spec described above, we use DROP IF EXISTS
        -- statements for all source-related tables. The IF EXISTS lets us
        -- handle both cases uniformly, doing "ideally" nothing in the type 2
        -- database, and for default databases, we drop only source-related
        -- tables from the database's "hdb_catalog" schema.
        | Bool
hdbMetadataTableExist -> do
            -- drop the event trigger functions from the table for default sources
            [(QualifiedTable, [TriggerName])]
-> ((QualifiedTable, [TriggerName]) -> TxET QErr m ())
-> TxET QErr m ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ (HashMap QualifiedTable [TriggerName]
-> [(QualifiedTable, [TriggerName])]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList TableEventTriggers ('Postgres pgKind)
HashMap QualifiedTable [TriggerName]
tableTriggersMap) (((QualifiedTable, [TriggerName]) -> TxET QErr m ())
 -> TxET QErr m ())
-> ((QualifiedTable, [TriggerName]) -> TxET QErr m ())
-> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ \(QualifiedTable
_table, [TriggerName]
triggers) ->
              [TriggerName] -> (TriggerName -> TxET QErr m ()) -> TxET QErr m ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ [TriggerName]
triggers ((TriggerName -> TxET QErr m ()) -> TxET QErr m ())
-> (TriggerName -> TxET QErr m ()) -> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ \TriggerName
triggerName ->
                TxET QErr IO () -> TxET QErr m ()
forall a. TxE QErr a -> TxET QErr m a
forall (m :: * -> *) a. MonadTx m => TxE QErr a -> m a
liftTx (TxET QErr IO () -> TxET QErr m ())
-> TxET QErr IO () -> TxET QErr m ()
forall a b. (a -> b) -> a -> b
$ TriggerName -> TxET QErr IO ()
dropTriggerQ TriggerName
triggerName
            (PGTxErr -> QErr) -> Query -> TxET QErr m ()
forall (m :: * -> *) a e.
(MonadIO m, FromRes a) =>
(PGTxErr -> e) -> Query -> TxET e m a
PG.multiQE
              PGTxErr -> QErr
defaultTxErrorHandler
              $(makeRelativeToProject "src-rsr/drop_pg_source.sql" >>= PG.sqlFromFile)
        -- Otherwise, we have a non-default postgres source, which has no metadata tables.
        -- We drop the entire "hdb_catalog" schema as discussed above.
        | Bool
otherwise ->
            TxET QErr m ()
forall (m :: * -> *). MonadTx m => m ()
dropHdbCatalogSchema

  -- Destory postgres source connection
  IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ PGExecCtx -> IO ()
_pecDestroyConnections (PGSourceConfig -> PGExecCtx
_pscExecCtx SourceConfig ('Postgres pgKind)
PGSourceConfig
sourceConfig)

  -- Run other drop hooks configured at source creation time
  IO () -> m ()
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO () -> m ()) -> IO () -> m ()
forall a b. (a -> b) -> a -> b
$ PGSourceConfig -> IO ()
_pscPostDropHook SourceConfig ('Postgres pgKind)
PGSourceConfig
sourceConfig