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

-- | MSSQL Source
--
-- Implements the Source related methods of the
-- 'Hasura.RQL.Types.Metadata.Backend.BackendMetadata' type class
-- for the MSSQL backend, which provides an interface for identifying the
-- MSSQL database instance (source) and manipulate it.
--
-- The actual instance is defined in "Hasura.Backends.MSSQL.Instances.Metadata".
module Hasura.Backends.MSSQL.DDL.Source
  ( resolveSourceConfig,
    resolveDatabaseMetadata,
    postDropSourceHook,
    prepareCatalog,
  )
where

import Control.Monad.Trans.Control (MonadBaseControl)
import Data.Environment qualified as Env
import Data.FileEmbed (makeRelativeToProject)
import Data.HashMap.Strict qualified as HashMap
import Data.Text.Lazy qualified as LT
import Database.MSSQL.Transaction
import Database.MSSQL.Transaction qualified as Tx
import Database.ODBC.SQLServer
import Database.ODBC.TH qualified as ODBC
import Hasura.Backends.MSSQL.Connection
import Hasura.Backends.MSSQL.DDL.EventTrigger
import Hasura.Backends.MSSQL.DDL.Source.Version
import Hasura.Backends.MSSQL.Instances.Types ()
import Hasura.Backends.MSSQL.Meta
import Hasura.Backends.MSSQL.SQL.Error qualified as HGE
import Hasura.Backends.MSSQL.Types
import Hasura.Base.Error
import Hasura.Prelude
import Hasura.RQL.Types.Backend (BackendConfig)
import Hasura.RQL.Types.BackendType
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.EventTrigger (RecreateEventTriggers (..))
import Hasura.RQL.Types.Source
import Hasura.Server.Migrate.Version (SourceCatalogMigrationState (..))
import Hasura.Server.Migrate.Version qualified as Version
import Hasura.Table.Cache
import Language.Haskell.TH.Lib qualified as TH
import Language.Haskell.TH.Syntax qualified as TH
import Text.Shakespeare.Text qualified as ST

resolveSourceConfig ::
  (MonadIO m, MonadResolveSource m) =>
  SourceName ->
  MSSQLConnConfiguration ->
  BackendSourceKind 'MSSQL ->
  BackendConfig 'MSSQL ->
  Env.Environment ->
  manager ->
  m (Either QErr MSSQLSourceConfig)
resolveSourceConfig :: forall (m :: * -> *) manager.
(MonadIO m, MonadResolveSource m) =>
SourceName
-> MSSQLConnConfiguration
-> BackendSourceKind 'MSSQL
-> BackendConfig 'MSSQL
-> Environment
-> manager
-> m (Either QErr MSSQLSourceConfig)
resolveSourceConfig SourceName
name MSSQLConnConfiguration
config BackendSourceKind 'MSSQL
_backendKind BackendConfig 'MSSQL
_backendConfig Environment
env manager
_manager = ExceptT QErr m MSSQLSourceConfig
-> m (Either QErr MSSQLSourceConfig)
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT do
  Environment
-> SourceName
-> MSSQLConnConfiguration
-> IO (Either QErr MSSQLSourceConfig)
sourceResolver <- ExceptT
  QErr
  m
  (Environment
   -> SourceName
   -> MSSQLConnConfiguration
   -> IO (Either QErr MSSQLSourceConfig))
ExceptT QErr m (SourceResolver 'MSSQL)
forall (m :: * -> *).
MonadResolveSource m =>
m (SourceResolver 'MSSQL)
getMSSQLSourceResolver
  ExceptT QErr m (Either QErr MSSQLSourceConfig)
-> ExceptT QErr m MSSQLSourceConfig
forall e (m :: * -> *) a. MonadError e m => m (Either e a) -> m a
liftEitherM (ExceptT QErr m (Either QErr MSSQLSourceConfig)
 -> ExceptT QErr m MSSQLSourceConfig)
-> ExceptT QErr m (Either QErr MSSQLSourceConfig)
-> ExceptT QErr m MSSQLSourceConfig
forall a b. (a -> b) -> a -> b
$ IO (Either QErr MSSQLSourceConfig)
-> ExceptT QErr m (Either QErr MSSQLSourceConfig)
forall a. IO a -> ExceptT QErr m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Either QErr MSSQLSourceConfig)
 -> ExceptT QErr m (Either QErr MSSQLSourceConfig))
-> IO (Either QErr MSSQLSourceConfig)
-> ExceptT QErr m (Either QErr MSSQLSourceConfig)
forall a b. (a -> b) -> a -> b
$ Environment
-> SourceName
-> MSSQLConnConfiguration
-> IO (Either QErr MSSQLSourceConfig)
sourceResolver Environment
env SourceName
name MSSQLConnConfiguration
config

resolveDatabaseMetadata ::
  (MonadIO m, MonadBaseControl IO m) =>
  MSSQLSourceConfig ->
  m (Either QErr (DBObjectsIntrospection 'MSSQL))
resolveDatabaseMetadata :: forall (m :: * -> *).
(MonadIO m, MonadBaseControl IO m) =>
MSSQLSourceConfig
-> m (Either QErr (DBObjectsIntrospection 'MSSQL))
resolveDatabaseMetadata MSSQLSourceConfig
config = ExceptT QErr m (DBObjectsIntrospection 'MSSQL)
-> m (Either QErr (DBObjectsIntrospection 'MSSQL))
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT do
  HashMap TableName (DBTableMetadata 'MSSQL)
dbTablesMetadata <- MSSQLExecCtx -> MSSQLRunTx
mssqlRunReadOnly MSSQLExecCtx
mssqlExecCtx (TxET QErr m (HashMap TableName (DBTableMetadata 'MSSQL))
 -> ExceptT QErr m (HashMap TableName (DBTableMetadata 'MSSQL)))
-> TxET QErr m (HashMap TableName (DBTableMetadata 'MSSQL))
-> ExceptT QErr m (HashMap TableName (DBTableMetadata 'MSSQL))
forall a b. (a -> b) -> a -> b
$ TxET QErr m (DBTablesMetadata 'MSSQL)
TxET QErr m (HashMap TableName (DBTableMetadata 'MSSQL))
forall (m :: * -> *).
MonadIO m =>
TxET QErr m (DBTablesMetadata 'MSSQL)
loadDBMetadata
  DBObjectsIntrospection 'MSSQL
-> ExceptT QErr m (DBObjectsIntrospection 'MSSQL)
forall a. a -> ExceptT QErr m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (DBObjectsIntrospection 'MSSQL
 -> ExceptT QErr m (DBObjectsIntrospection 'MSSQL))
-> DBObjectsIntrospection 'MSSQL
-> ExceptT QErr m (DBObjectsIntrospection 'MSSQL)
forall a b. (a -> b) -> a -> b
$ DBTablesMetadata 'MSSQL
-> DBFunctionsMetadata 'MSSQL
-> ScalarMap 'MSSQL
-> LogicalModels 'MSSQL
-> DBObjectsIntrospection 'MSSQL
forall (b :: BackendType).
DBTablesMetadata b
-> DBFunctionsMetadata b
-> ScalarMap b
-> LogicalModels b
-> DBObjectsIntrospection b
DBObjectsIntrospection DBTablesMetadata 'MSSQL
HashMap TableName (DBTableMetadata 'MSSQL)
dbTablesMetadata DBFunctionsMetadata 'MSSQL
HashMap FunctionName (FunctionOverloads 'MSSQL)
forall a. Monoid a => a
mempty ScalarMap 'MSSQL
forall a. Monoid a => a
mempty LogicalModels 'MSSQL
forall a. Monoid a => a
mempty
  where
    MSSQLSourceConfig ConnectionString
_connString MSSQLExecCtx
mssqlExecCtx Int
_numReadReplicas = MSSQLSourceConfig
config

postDropSourceHook ::
  (MonadIO m, MonadBaseControl IO m) =>
  MSSQLSourceConfig ->
  TableEventTriggers 'MSSQL ->
  m ()
postDropSourceHook :: forall (m :: * -> *).
(MonadIO m, MonadBaseControl IO m) =>
MSSQLSourceConfig -> TableEventTriggers 'MSSQL -> m ()
postDropSourceHook (MSSQLSourceConfig ConnectionString
_ MSSQLExecCtx
mssqlExecCtx Int
_) TableEventTriggers 'MSSQL
tableTriggersMap = do
  -- The SQL triggers for MSSQL source are created within the schema of the table,
  -- and is not associated with 'hdb_catalog' schema. Thus only deleting the
  -- 'hdb_catalog' schema is not sufficient, since it will still leave the SQL
  -- triggers within the table schema.
  --
  -- This causes problems, whenever the next insert/delete/update operation occurs
  -- the SQL triggers will try to unsuccessfully insert event_log to the nonexistent
  -- 'hdb_catalog.event_log' table. The left over SQL triggers thus stops any
  -- operation that will happen on the table.
  --
  -- Hence we first delete all the related Hasura SQL triggers and then drop the
  -- 'hdb_catalog' schema.
  [(TableName, [TriggerName])]
-> ((TableName, [TriggerName]) -> m ()) -> m ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ (HashMap TableName [TriggerName] -> [(TableName, [TriggerName])]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList TableEventTriggers 'MSSQL
HashMap TableName [TriggerName]
tableTriggersMap) (((TableName, [TriggerName]) -> m ()) -> m ())
-> ((TableName, [TriggerName]) -> m ()) -> m ()
forall a b. (a -> b) -> a -> b
$ \(_table :: TableName
_table@(TableName Text
_tableName SchemaName
schema), [TriggerName]
triggers) ->
    [TriggerName] -> (TriggerName -> m (Either QErr ())) -> m ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
t a -> (a -> f b) -> f ()
for_ [TriggerName]
triggers ((TriggerName -> m (Either QErr ())) -> m ())
-> (TriggerName -> m (Either QErr ())) -> m ()
forall a b. (a -> b) -> a -> b
$ \TriggerName
triggerName ->
      IO (Either QErr ()) -> m (Either QErr ())
forall a. IO a -> m a
forall (m :: * -> *) a. MonadIO m => IO a -> m a
liftIO (IO (Either QErr ()) -> m (Either QErr ()))
-> IO (Either QErr ()) -> m (Either QErr ())
forall a b. (a -> b) -> a -> b
$ ExceptT QErr IO () -> IO (Either QErr ())
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT QErr IO () -> IO (Either QErr ()))
-> ExceptT QErr IO () -> IO (Either QErr ())
forall a b. (a -> b) -> a -> b
$ MSSQLExecCtx -> MSSQLRunTx
mssqlRunReadWrite MSSQLExecCtx
mssqlExecCtx (TriggerName -> SchemaName -> TxET QErr IO ()
dropTriggerQ TriggerName
triggerName SchemaName
schema)
  Either QErr ()
_ <- ExceptT QErr m () -> m (Either QErr ())
forall e (m :: * -> *) a. ExceptT e m a -> m (Either e a)
runExceptT (ExceptT QErr m () -> m (Either QErr ()))
-> ExceptT QErr m () -> m (Either QErr ())
forall a b. (a -> b) -> a -> b
$ MSSQLExecCtx -> MSSQLRunTx
mssqlRunReadWrite MSSQLExecCtx
mssqlExecCtx TxET QErr m ()
forall (m :: * -> *). MonadMSSQLTx m => m ()
dropSourceCatalog
  -- Close the 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
$ MSSQLExecCtx -> IO ()
mssqlDestroyConn MSSQLExecCtx
mssqlExecCtx

doesSchemaExist :: (MonadMSSQLTx m) => SchemaName -> m Bool
doesSchemaExist :: forall (m :: * -> *). MonadMSSQLTx m => SchemaName -> m Bool
doesSchemaExist (SchemaName Text
schemaName) = do
  TxE QErr Bool -> m Bool
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadMSSQLTx m => TxE QErr a -> m a
liftMSSQLTx
    (TxE QErr Bool -> m Bool) -> TxE QErr Bool -> m Bool
forall a b. (a -> b) -> a -> b
$ (MSSQLTxError -> QErr) -> Query -> TxE QErr Bool
forall (m :: * -> *) a e.
(MonadIO m, FromRow a) =>
(MSSQLTxError -> e) -> Query -> TxET e m a
Tx.singleRowQueryE
      MSSQLTxError -> QErr
HGE.defaultMSSQLTxErrorHandler
      [ODBC.sql|
      SELECT CAST (
        CASE
          WHEN EXISTS( SELECT 1 FROM sys.schemas WHERE name = $schemaName )
            THEN 1
          ELSE 0
        END
      AS BIT)
    |]

doesTableExist :: (MonadMSSQLTx m) => TableName -> m Bool
doesTableExist :: forall (m :: * -> *). MonadMSSQLTx m => TableName -> m Bool
doesTableExist TableName
tableName = do
  TxE QErr Bool -> m Bool
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadMSSQLTx m => TxE QErr a -> m a
liftMSSQLTx
    (TxE QErr Bool -> m Bool) -> TxE QErr Bool -> m Bool
forall a b. (a -> b) -> a -> b
$ (MSSQLTxError -> QErr) -> Query -> TxE QErr Bool
forall (m :: * -> *) a e.
(MonadIO m, FromRow a) =>
(MSSQLTxError -> e) -> Query -> TxET e m a
Tx.singleRowQueryE
      MSSQLTxError -> QErr
HGE.defaultMSSQLTxErrorHandler
      [ODBC.sql|
        SELECT CAST (
          CASE
            WHEN (Select OBJECT_ID($qualifiedTable)) IS NOT NULL
              THEN 1
            ELSE 0
          END
        AS BIT)
      |]
  where
    qualifiedTable :: Text
qualifiedTable = TableName -> Text
qualifyTableName TableName
tableName

-- | Initialise catalog tables for a source, including those required by the event delivery subsystem.
prepareCatalog ::
  (MonadIO m, MonadBaseControl IO m) =>
  MSSQLSourceConfig ->
  ExceptT QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
prepareCatalog :: forall (m :: * -> *).
(MonadIO m, MonadBaseControl IO m) =>
MSSQLSourceConfig
-> ExceptT
     QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
prepareCatalog MSSQLSourceConfig
sourceConfig = MSSQLExecCtx -> MSSQLRunTx
mssqlRunSerializableTx (MSSQLSourceConfig -> MSSQLExecCtx
_mscExecCtx MSSQLSourceConfig
sourceConfig) do
  Bool
hdbCatalogExist <- SchemaName -> TxET QErr m Bool
forall (m :: * -> *). MonadMSSQLTx m => SchemaName -> m Bool
doesSchemaExist SchemaName
"hdb_catalog"
  Bool
eventLogTableExist <- TableName -> TxET QErr m Bool
forall (m :: * -> *). MonadMSSQLTx m => TableName -> m Bool
doesTableExist (TableName -> TxET QErr m Bool) -> TableName -> TxET QErr m Bool
forall a b. (a -> b) -> a -> b
$ Text -> SchemaName -> TableName
TableName Text
"event_log" SchemaName
"hdb_catalog"
  Bool
sourceVersionTableExist <- TableName -> TxET QErr m Bool
forall (m :: * -> *). MonadMSSQLTx m => TableName -> m Bool
doesTableExist (TableName -> TxET QErr m Bool) -> TableName -> TxET QErr m Bool
forall a b. (a -> b) -> a -> b
$ Text -> SchemaName -> TableName
TableName Text
"hdb_source_catalog_version" SchemaName
"hdb_catalog"
  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. MonadMSSQLTx m => TxE QErr a -> m a
liftMSSQLTx do
        (MSSQLTxError -> QErr) -> Query -> TxET QErr IO ()
forall (m :: * -> *) e.
MonadIO m =>
(MSSQLTxError -> e) -> Query -> TxET e m ()
unitQueryE MSSQLTxError -> QErr
HGE.defaultMSSQLTxErrorHandler Query
"CREATE SCHEMA hdb_catalog"
        TxET QErr IO ()
initSourceCatalog
        (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 'MSSQL -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion 'MSSQL
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. MonadMSSQLTx m => TxE QErr a -> m a
liftMSSQLTx TxET QErr IO ()
initSourceCatalog
        (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 'MSSQL -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion 'MSSQL
latestSourceCatalogVersion)
    | Bool
otherwise -> TxET QErr m (RecreateEventTriggers, SourceCatalogMigrationState)
forall (m :: * -> *).
MonadMSSQLTx m =>
m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalog
  where
    initSourceCatalog :: TxET QErr IO ()
initSourceCatalog = do
      (MSSQLTxError -> QErr) -> Query -> TxET QErr IO ()
forall (m :: * -> *) e.
MonadIO m =>
(MSSQLTxError -> e) -> Query -> TxET e m ()
unitQueryE MSSQLTxError -> QErr
HGE.defaultMSSQLTxErrorHandler $(makeRelativeToProject "src-rsr/mssql/init_mssql_source.sql" >>= ODBC.sqlFile)
      SourceCatalogVersion 'MSSQL -> TxET QErr IO ()
forall (m :: * -> *).
MonadMSSQLTx m =>
SourceCatalogVersion 'MSSQL -> m ()
setSourceCatalogVersion SourceCatalogVersion 'MSSQL
latestSourceCatalogVersion

dropSourceCatalog :: (MonadMSSQLTx m) => m ()
dropSourceCatalog :: forall (m :: * -> *). MonadMSSQLTx m => m ()
dropSourceCatalog = do
  let sql :: Query
sql = $(makeRelativeToProject "src-rsr/mssql/drop_mssql_source.sql" >>= ODBC.sqlFile)
  TxET QErr IO () -> m ()
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadMSSQLTx m => TxE QErr a -> m a
liftMSSQLTx (TxET QErr IO () -> m ()) -> TxET QErr IO () -> m ()
forall a b. (a -> b) -> a -> b
$ (MSSQLTxError -> QErr) -> Query -> TxET QErr IO ()
forall (m :: * -> *) e.
MonadIO m =>
(MSSQLTxError -> e) -> Query -> TxET e m ()
unitQueryE MSSQLTxError -> QErr
HGE.defaultMSSQLTxErrorHandler Query
sql

migrateSourceCatalog :: (MonadMSSQLTx m) => m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalog :: forall (m :: * -> *).
MonadMSSQLTx m =>
m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalog =
  m (SourceCatalogVersion 'MSSQL)
forall (m :: * -> *).
MonadMSSQLTx m =>
m (SourceCatalogVersion 'MSSQL)
getSourceCatalogVersion m (SourceCatalogVersion 'MSSQL)
-> (SourceCatalogVersion 'MSSQL
    -> 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 'MSSQL
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall (m :: * -> *).
MonadMSSQLTx m =>
SourceCatalogVersion 'MSSQL
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalogFrom

migrateSourceCatalogFrom :: (MonadMSSQLTx m) => SourceCatalogVersion -> m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalogFrom :: forall (m :: * -> *).
MonadMSSQLTx m =>
SourceCatalogVersion 'MSSQL
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
migrateSourceCatalogFrom SourceCatalogVersion 'MSSQL
prevVersion
  | SourceCatalogVersion 'MSSQL
prevVersion SourceCatalogVersion 'MSSQL -> SourceCatalogVersion 'MSSQL -> Bool
forall a. Eq a => a -> a -> Bool
== SourceCatalogVersion 'MSSQL
latestSourceCatalogVersion = (RecreateEventTriggers, SourceCatalogMigrationState)
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (RecreateEventTriggers
RETDoNothing, Int -> SourceCatalogMigrationState
SCMSNothingToDo (Int -> SourceCatalogMigrationState)
-> Int -> SourceCatalogMigrationState
forall a b. (a -> b) -> a -> b
$ SourceCatalogVersion 'MSSQL -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion 'MSSQL
latestSourceCatalogVersion)
  | [] <- [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
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 'MSSQL -> Text
forall a. Show a => a -> Text
tshow SourceCatalogVersion 'MSSQL
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 'MSSQL -> Text
forall a. Show a => a -> Text
tshow SourceCatalogVersion 'MSSQL
prevVersion
  | Bool
otherwise = do
      TxET QErr IO () -> m ()
forall a. TxE QErr a -> m a
forall (m :: * -> *) a. MonadMSSQLTx m => TxE QErr a -> m a
liftMSSQLTx (TxET QErr IO () -> m ()) -> TxET QErr IO () -> m ()
forall a b. (a -> b) -> a -> b
$ ((SourceCatalogVersion 'MSSQL, TxE QErr [Text]) -> TxE QErr [Text])
-> [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
-> TxET QErr IO ()
forall (t :: * -> *) (f :: * -> *) a b.
(Foldable t, Applicative f) =>
(a -> f b) -> t a -> f ()
traverse_ (SourceCatalogVersion 'MSSQL, TxE QErr [Text]) -> TxE QErr [Text]
forall a b. (a, b) -> b
snd [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
neededMigrations
      SourceCatalogVersion 'MSSQL -> m ()
forall (m :: * -> *).
MonadMSSQLTx m =>
SourceCatalogVersion 'MSSQL -> m ()
setSourceCatalogVersion SourceCatalogVersion 'MSSQL
latestSourceCatalogVersion
      (RecreateEventTriggers, SourceCatalogMigrationState)
-> m (RecreateEventTriggers, SourceCatalogMigrationState)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (RecreateEventTriggers
RETRecreate, Int -> Int -> SourceCatalogMigrationState
SCMSMigratedTo (SourceCatalogVersion 'MSSQL -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion 'MSSQL
prevVersion) (SourceCatalogVersion 'MSSQL -> Int
forall (backend :: BackendType).
SourceCatalogVersion backend -> Int
Version.unSourceCatalogVersion SourceCatalogVersion 'MSSQL
latestSourceCatalogVersion))
  where
    neededMigrations :: [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
neededMigrations =
      ((SourceCatalogVersion 'MSSQL, TxE QErr [Text]) -> Bool)
-> [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
-> [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
forall a. (a -> Bool) -> [a] -> [a]
dropWhile ((SourceCatalogVersion 'MSSQL -> SourceCatalogVersion 'MSSQL -> Bool
forall a. Eq a => a -> a -> Bool
/= SourceCatalogVersion 'MSSQL
prevVersion) (SourceCatalogVersion 'MSSQL -> Bool)
-> ((SourceCatalogVersion 'MSSQL, TxE QErr [Text])
    -> SourceCatalogVersion 'MSSQL)
-> (SourceCatalogVersion 'MSSQL, TxE QErr [Text])
-> Bool
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (SourceCatalogVersion 'MSSQL, TxE QErr [Text])
-> SourceCatalogVersion 'MSSQL
forall a b. (a, b) -> a
fst) [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
sourceMigrations

sourceMigrations :: [(SourceCatalogVersion, TxE QErr [Text])]
sourceMigrations :: [(SourceCatalogVersion 'MSSQL, TxE QErr [Text])]
sourceMigrations =
  $( let migrationFromFile from =
           let to = succ from
               path = "src-rsr/mssql/mssql_source_migrations/" <> show from <> "_to_" <> show to <> ".sql"
            in do
                 [|(multiRowQueryE HGE.defaultMSSQLTxErrorHandler $ rawUnescapedText . LT.toStrict $ $(makeRelativeToProject path >>= ST.stextFile))|]

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