2021-02-23 20:37:27 +03:00
{- # OPTIONS_GHC - fno - warn - orphans # -}
2021-05-21 14:37:34 +03:00
module Hasura.Backends.MSSQL.Instances.Execute
2021-09-24 01:56:37 +03:00
( MultiplexedQuery' ( .. ) ,
2021-05-21 14:37:34 +03:00
multiplexRootReselect ,
)
2021-09-24 01:56:37 +03:00
where
2021-10-22 17:49:15 +03:00
import Control.Monad.Trans.Control ( MonadBaseControl )
2021-10-01 15:52:19 +03:00
import Control.Monad.Validate qualified as V
2021-09-24 01:56:37 +03:00
import Data.Aeson.Extended qualified as J
import Data.HashMap.Strict qualified as Map
import Data.HashSet qualified as Set
import Data.List.NonEmpty qualified as NE
import Data.Text.Extended qualified as T
2021-10-01 15:52:19 +03:00
import Database.MSSQL.Transaction qualified as Tx
import Database.ODBC.Internal qualified as ODBCI
2021-09-24 01:56:37 +03:00
import Database.ODBC.SQLServer qualified as ODBC
import Hasura.Backends.MSSQL.Connection
import Hasura.Backends.MSSQL.FromIr as TSQL
import Hasura.Backends.MSSQL.Plan
import Hasura.Backends.MSSQL.SQL.Value ( txtEncodedColVal )
2021-10-01 15:52:19 +03:00
import Hasura.Backends.MSSQL.ToQuery as TQ
2021-09-24 01:56:37 +03:00
import Hasura.Backends.MSSQL.Types as TSQL
import Hasura.Base.Error
import Hasura.EncJSON
import Hasura.GraphQL.Execute.Backend
import Hasura.GraphQL.Execute.LiveQuery.Plan
import Hasura.GraphQL.Parser
import Hasura.Prelude
import Hasura.RQL.IR
2021-10-01 15:52:19 +03:00
import Hasura.RQL.IR qualified as IR
2021-09-24 01:56:37 +03:00
import Hasura.RQL.Types
import Hasura.RQL.Types qualified as RQLTypes
import Hasura.RQL.Types.Column qualified as RQLColumn
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.Session
import Language.GraphQL.Draft.Syntax qualified as G
2021-02-23 20:37:27 +03:00
instance BackendExecute 'MSSQL where
2021-09-24 01:56:37 +03:00
type PreparedQuery 'MSSQL = Text
2021-04-20 19:57:14 +03:00
type MultiplexedQuery 'MSSQL = MultiplexedQuery'
2021-09-24 01:56:37 +03:00
type ExecutionMonad 'MSSQL = ExceptT QErr IO
2021-02-23 20:37:27 +03:00
mkDBQueryPlan = msDBQueryPlan
mkDBMutationPlan = msDBMutationPlan
mkDBSubscriptionPlan = msDBSubscriptionPlan
2021-04-13 14:10:08 +03:00
mkDBQueryExplain = msDBQueryExplain
mkLiveQueryExplain = msDBLiveQueryExplain
2021-02-23 20:37:27 +03:00
2021-09-22 13:43:05 +03:00
-- NOTE: Currently unimplemented!.
--
-- This function is just a stub for future implementation; for now it just
-- throws a 500 error.
mkDBRemoteRelationshipPlan =
msDBRemoteRelationshipPlan
2021-06-01 13:04:29 +03:00
-- Multiplexed query
2021-05-21 14:37:34 +03:00
2021-04-20 19:57:14 +03:00
newtype MultiplexedQuery' = MultiplexedQuery' Reselect
2021-02-23 20:37:27 +03:00
2021-04-20 19:57:14 +03:00
instance T . ToTxt MultiplexedQuery' where
toTxt ( MultiplexedQuery' reselect ) = T . toTxt $ toQueryPretty $ fromReselect reselect
2021-02-23 20:37:27 +03:00
2021-06-01 13:04:29 +03:00
-- Query
2021-02-23 20:37:27 +03:00
2021-09-24 01:56:37 +03:00
msDBQueryPlan ::
forall m .
( MonadError QErr m
) =>
UserInfo ->
SourceName ->
SourceConfig 'MSSQL ->
QueryDB 'MSSQL ( Const Void ) ( UnpreparedValue 'MSSQL ) ->
m ( DBStepInfo 'MSSQL )
2021-09-23 15:37:56 +03:00
msDBQueryPlan userInfo sourceName sourceConfig qrf = do
2021-07-29 11:29:12 +03:00
-- TODO (naveen): Append Query Tags to the query
2021-05-21 14:37:34 +03:00
let sessionVariables = _uiSession userInfo
2021-06-01 13:04:29 +03:00
statement <- planQuery sessionVariables qrf
let printer = fromSelect statement
2021-05-21 14:37:34 +03:00
queryString = ODBC . renderQuery $ toQueryPretty printer
2021-09-24 01:56:37 +03:00
pool = _mscConnectionPool sourceConfig
2021-05-21 14:37:34 +03:00
odbcQuery = encJFromText <$> runJSONPathQuery pool ( toQueryFlat printer )
2021-10-01 15:52:19 +03:00
pure $ DBStepInfo @ 'MSSQL sourceName sourceConfig ( Just queryString ) odbcQuery
2021-02-23 20:37:27 +03:00
2021-09-24 01:56:37 +03:00
runShowplan ::
ODBC . Query -> ODBC . Connection -> IO [ Text ]
2021-05-11 13:04:38 +03:00
runShowplan query conn = do
ODBC . exec conn " SET SHOWPLAN_TEXT ON "
texts <- ODBC . query conn query
ODBC . exec conn " SET SHOWPLAN_TEXT OFF "
-- we don't need to use 'finally' here - if an exception occurs,
-- the connection is removed from the resource pool in 'withResource'.
pure texts
2021-09-24 01:56:37 +03:00
msDBQueryExplain ::
MonadError QErr m =>
G . Name ->
UserInfo ->
SourceName ->
SourceConfig 'MSSQL ->
QueryDB 'MSSQL ( Const Void ) ( UnpreparedValue 'MSSQL ) ->
m ( AB . AnyBackend DBStepInfo )
2021-04-13 14:10:08 +03:00
msDBQueryExplain fieldName userInfo sourceName sourceConfig qrf = do
2021-05-21 14:37:34 +03:00
let sessionVariables = _uiSession userInfo
2021-06-01 13:04:29 +03:00
statement <- planQuery sessionVariables qrf
2021-09-24 01:56:37 +03:00
let query = toQueryPretty ( fromSelect statement )
2021-10-01 15:52:19 +03:00
queryString = ODBC . renderQuery query
2021-09-24 01:56:37 +03:00
pool = _mscConnectionPool sourceConfig
odbcQuery =
2021-05-11 13:04:38 +03:00
withMSSQLPool
pool
2021-10-22 17:49:15 +03:00
( \ conn -> liftIO do
2021-09-24 01:56:37 +03:00
showplan <- runShowplan query conn
pure
( encJFromJValue $
ExplainPlan
fieldName
( Just queryString )
( Just showplan )
)
)
pure $
AB . mkAnyBackend $
DBStepInfo @ 'MSSQL sourceName sourceConfig Nothing odbcQuery
msDBLiveQueryExplain ::
2021-10-22 17:49:15 +03:00
( MonadIO m , MonadBaseControl IO m , MonadError QErr m ) =>
2021-09-24 01:56:37 +03:00
LiveQueryPlan 'MSSQL ( MultiplexedQuery 'MSSQL ) ->
m LiveQueryPlanExplanation
2021-05-11 13:04:38 +03:00
msDBLiveQueryExplain ( LiveQueryPlan plan sourceConfig variables ) = do
let ( MultiplexedQuery' reselect ) = _plqpQuery plan
query = toQueryPretty $ fromSelect $ multiplexRootReselect [ ( dummyCohortId , variables ) ] reselect
pool = _mscConnectionPool sourceConfig
2021-10-22 17:49:15 +03:00
explainInfo <- withMSSQLPool pool ( liftIO . runShowplan query )
2021-05-11 13:04:38 +03:00
pure $ LiveQueryPlanExplanation ( T . toTxt query ) explainInfo variables
2021-04-20 19:57:14 +03:00
--------------------------------------------------------------------------------
-- Producing the correct SQL-level list comprehension to multiplex a query
-- Problem description:
--
-- Generate a query that repeats the same query N times but with
-- certain slots replaced:
--
-- [ Select x y | (x,y) <- [..] ]
--
2021-09-24 01:56:37 +03:00
multiplexRootReselect ::
[ ( CohortId , CohortVariables ) ] ->
TSQL . Reselect ->
TSQL . Select
2021-04-20 19:57:14 +03:00
multiplexRootReselect variables rootReselect =
2021-10-01 15:52:19 +03:00
emptySelect
2021-09-24 01:56:37 +03:00
{ selectTop = NoTop ,
selectProjections =
2021-04-20 19:57:14 +03:00
[ FieldNameProjection
Aliased
{ aliasedThing =
TSQL . FieldName
2021-09-24 01:56:37 +03:00
{ fieldNameEntity = rowAlias ,
fieldName = resultIdAlias
} ,
aliasedAlias = resultIdAlias
} ,
ExpressionProjection
2021-04-20 19:57:14 +03:00
Aliased
{ aliasedThing =
ColumnExpression
2021-09-24 01:56:37 +03:00
( TSQL . FieldName
{ fieldNameEntity = resultAlias ,
fieldName = TSQL . jsonFieldName
}
) ,
aliasedAlias = resultAlias
2021-04-20 19:57:14 +03:00
}
2021-09-24 01:56:37 +03:00
] ,
selectFrom =
Just $
FromOpenJson
Aliased
{ aliasedThing =
OpenJson
{ openJsonExpression =
ValueExpression ( ODBC . TextValue $ lbsToTxt $ J . encode variables ) ,
openJsonWith =
Just $
NE . fromList
[ UuidField resultIdAlias ( Just $ IndexPath RootPath 0 ) ,
JsonField resultVarsAlias ( Just $ IndexPath RootPath 1 )
]
} ,
aliasedAlias = rowAlias
} ,
selectJoins =
2021-04-20 19:57:14 +03:00
[ Join
2021-09-24 01:56:37 +03:00
{ joinSource = JoinReselect rootReselect ,
joinJoinAlias =
2021-04-20 19:57:14 +03:00
JoinAlias
2021-09-24 01:56:37 +03:00
{ joinAliasEntity = resultAlias ,
joinAliasField = Just TSQL . jsonFieldName
2021-04-20 19:57:14 +03:00
}
}
2021-09-24 01:56:37 +03:00
] ,
selectWhere = Where mempty ,
selectFor =
JsonFor ForJson { jsonCardinality = JsonArray , jsonRoot = NoRoot } ,
selectOrderBy = Nothing ,
selectOffset = Nothing
2021-04-20 19:57:14 +03:00
}
2021-02-23 20:37:27 +03:00
-- mutation
2021-09-24 01:56:37 +03:00
msDBMutationPlan ::
forall m .
( MonadError QErr m
) =>
UserInfo ->
Bool ->
SourceName ->
SourceConfig 'MSSQL ->
MutationDB 'MSSQL ( Const Void ) ( UnpreparedValue 'MSSQL ) ->
m ( DBStepInfo 'MSSQL )
2021-10-01 15:52:19 +03:00
msDBMutationPlan userInfo stringifyNum sourceName sourceConfig mrf = do
go <$> case mrf of
MDBInsert annInsert -> executeInsert userInfo stringifyNum sourceConfig annInsert
MDBUpdate _annUpdate -> throw400 NotSupported " update mutations are not supported in MSSQL "
MDBDelete _annDelete -> throw400 NotSupported " delete mutations are not supported in MSSQL "
MDBFunction { } -> throw400 NotSupported " function mutations are not supported in MSSQL "
where
go v = DBStepInfo @ 'MSSQL sourceName sourceConfig Nothing v
-- | Execution of a MSSQL insert mutation broadly involves two steps.
--
-- -- insert_table(objects: [
-- -- {column1: value1, column2: value2},
-- -- {column1: value3, column2: value4}
-- -- ]
-- -- ){
-- -- affected_rows
-- -- returning {
-- -- column1
-- -- column2
-- -- }
-- -- }
-- --
-- Step 1: Inserting rows into the table
-- --
-- -- a. Generate an SQL Insert statement from the GraphQL insert mutation with OUTPUT expression to return
-- -- primary key column values after insertion.
-- -- b. Before insert, Set IDENTITY_INSERT to ON if any insert row contains atleast one identity column.
-- --
-- -- SET IDENTITY_INSERT some_table ON;
-- -- INSERT INTO some_table (column1, column2) OUTPUT INSERTED.pkey_column1, INSERTED.pkey_column2 VALUES (value1, value2), (value3, value4);
-- --
-- Step 2: Generation of the mutation response
-- --
-- -- An SQL statement is generated and when executed it returns the mutation selection set containing 'affected_rows' and 'returning' field values.
-- -- The statement is generated with multiple sub select queries explained below:
-- --
-- -- a. A SQL Select statement to fetch only inserted rows from the table using primary key column values fetched from
-- -- Step 1 in the WHERE clause
-- --
-- -- <table_select> :=
-- -- SELECT * FROM some_table WHERE (pkey_column1 = value1 AND pkey_column2 = value2) OR (pkey_column1 = value3 AND pkey_column2 = value4)
-- --
-- -- The above select statement is referred through a common table expression - "WITH [with_alias] AS (<table_select>)"
-- --
-- -- b. The 'affected_rows' field value is obtained by using COUNT aggregation and the 'returning' field selection set is translated to
-- -- a SQL select statement using @'mkSQLSelect'.
-- --
-- -- <mutation_output_select> :=
-- -- SELECT (SELECT COUNT(*) FROM [with_alias]) AS [affected_rows], (select_from_returning) AS [returning] FOR JSON PATH, INCLUDE_NULL_VALUES, WITHOUT_ARRAY_WRAPPER
-- --
-- -- c. Evaluate the check constraint using CASE expression. We use SUM aggregation to check if any inserted row has failed the check constraint.
-- --
-- -- <check_constraint_select> :=
-- -- SELECT SUM(CASE WHEN <check_boolean_expression> THEN 0 ELSE 1 END) FROM [with_alias]
-- --
-- -- d. The final select statement look like
-- --
-- -- WITH "with_alias" AS (<table_select>)
-- -- SELECT (<mutation_output_select>) AS [mutation_response], (<check_constraint_select>) AS [check_constraint_select]
-- --
-- -- When executed, the above statement returns a single row with mutation response as a string value and check constraint result as an integer value.
executeInsert ::
MonadError QErr m =>
UserInfo ->
Bool ->
SourceConfig 'MSSQL ->
AnnInsert 'MSSQL ( Const Void ) ( UnpreparedValue 'MSSQL ) ->
m ( ExceptT QErr IO EncJSON )
executeInsert userInfo stringifyNum sourceConfig annInsert = do
-- Convert the leaf values from @'UnpreparedValue' to sql @'Expression'
insert <- traverse ( prepareValueQuery sessionVariables ) annInsert
let insertTx = buildInsertTx insert
2021-10-22 17:49:15 +03:00
pure $ withMSSQLPool pool $ Tx . runTxE fromMSSQLTxError insertTx
2021-10-01 15:52:19 +03:00
where
sessionVariables = _uiSession userInfo
pool = _mscConnectionPool sourceConfig
table = _aiTableName $ _aiData annInsert
withSelectTableAlias = " t_ " <> tableName table
withAlias = " with_alias "
buildInsertTx :: AnnInsert 'MSSQL ( Const Void ) Expression -> Tx . TxET QErr IO EncJSON
buildInsertTx insert = do
let identityColumns = _mssqlIdentityColumns $ _aiExtraInsertData $ _aiData insert
insertColumns = concatMap ( map fst . getInsertColumns ) $ _aiInsObj $ _aiData insert
-- Set identity insert to ON if insert object contains identity columns
when ( any ( ` elem ` identityColumns ) insertColumns ) $
Tx . unitQueryE fromMSSQLTxError $
toQueryFlat $
TQ . fromSetIdentityInsert $
SetIdenityInsert ( _aiTableName $ _aiData insert ) SetON
-- Generate the INSERT query
let insertQuery = toQueryFlat $ TQ . fromInsert $ TSQL . fromInsert insert
fromODBCException e =
( err400 MSSQLError " insert query exception " ) { qeInternal = Just ( ExtraInternal $ odbcExceptionToJSONValue e ) }
-- Execute the INSERT query and fetch the primary key values
primaryKeyValues <- Tx . buildGenericTxE fromODBCException $ \ conn -> ODBCI . query conn ( ODBC . renderQuery insertQuery )
let withSelect = generateWithSelect primaryKeyValues
-- WITH [with_alias] AS (select_query)
withExpression = With $ pure $ Aliased withSelect withAlias
mutationOutputSelect <- mkMutationOutputSelect stringifyNum withAlias $ _aiOutput insert
let ( checkCondition , _ ) = _aiCheckCond $ _aiData insert
-- The check constraint is translated to boolean expression
checkBoolExp <-
V . runValidate ( runFromIr $ runReaderT ( fromGBoolExp checkCondition ) ( EntityAlias withAlias ) )
` onLeft ` ( throw500 . tshow )
-- SELECT (<mutation_output_select>) AS [mutation_response], (<check_constraint_select>) AS [check_constraint_select]
let mutationOutputCheckConstraintSelect = selectMutationOutputAndCheckCondition mutationOutputSelect checkBoolExp
-- WITH "with_alias" AS (<table_select>)
-- SELECT (<mutation_output_select>) AS [mutation_response], (<check_constraint_select>) AS [check_constraint_select]
finalSelect = mutationOutputCheckConstraintSelect { selectWith = Just withExpression }
( responseText , checkConditionInt ) <- Tx . singleRowQueryE fromMSSQLTxError ( toQueryFlat $ TQ . fromSelect finalSelect )
unless ( checkConditionInt == ( 0 :: Int ) ) $
throw400 PermissionError " check constraint of an insert permission has failed "
pure $ encJFromText responseText
columnFieldExpression :: ODBCI . Column -> Expression
columnFieldExpression column =
ColumnExpression $ TSQL . FieldName ( ODBCI . columnName column ) withSelectTableAlias
generateWithSelect :: [ [ ( ODBCI . Column , ODBC . Value ) ] ] -> Select
generateWithSelect pkeyValues =
emptySelect
{ selectProjections = [ StarProjection ] ,
selectFrom = Just $ FromQualifiedTable $ Aliased table withSelectTableAlias ,
selectWhere = whereExpression
}
where
-- WHERE (column1 = value1 AND column2 = value2) OR (column1 = value3 AND column2 = value4)
whereExpression =
let mkColumnEqExpression ( column , value ) =
OpExpression EQ' ( columnFieldExpression column ) ( ValueExpression value )
in Where $ pure $ OrExpression $ map ( AndExpression . map mkColumnEqExpression ) pkeyValues
generateCheckConstraintSelect :: Expression -> Select
generateCheckConstraintSelect checkBoolExp =
let zeroValue = ValueExpression $ ODBC . IntValue 0
oneValue = ValueExpression $ ODBC . IntValue 1
caseExpression = ConditionalExpression checkBoolExp zeroValue oneValue
sumAggregate = OpAggregate " SUM " [ caseExpression ]
in emptySelect
{ selectProjections = [ AggregateProjection ( Aliased sumAggregate " check " ) ] ,
selectFrom = Just $ TSQL . FromIdentifier withAlias
}
selectMutationOutputAndCheckCondition :: Select -> Expression -> Select
selectMutationOutputAndCheckCondition mutationOutputSelect checkBoolExp =
let mutationOutputProjection =
ExpressionProjection $ Aliased ( SelectExpression mutationOutputSelect ) " mutation_response "
checkConstraintProjection =
ExpressionProjection $ Aliased ( SelectExpression ( generateCheckConstraintSelect checkBoolExp ) ) " check_constraint_select "
in emptySelect { selectProjections = [ mutationOutputProjection , checkConstraintProjection ] }
-- | Generate a SQL SELECT statement which outputs the mutation response
--
-- For multi row inserts:
-- SELECT (SELECT COUNT(*) FROM [with_alias]) AS [affected_rows], (select_from_returning) AS [returning] FOR JSON PATH, INCLUDE_NULL_VALUES, WITHOUT_ARRAY_WRAPPER
--
-- For single row insert: the selection set is translated to SQL query using @'mkSQLSelect'
mkMutationOutputSelect ::
( MonadError QErr m ) =>
Bool ->
Text ->
MutationOutputG 'MSSQL ( Const Void ) Expression ->
m Select
mkMutationOutputSelect stringifyNum withAlias = \ case
IR . MOutMultirowFields multiRowFields -> do
projections <- forM multiRowFields $ \ ( fieldName , field' ) -> do
let mkProjection = ExpressionProjection . flip Aliased ( getFieldNameTxt fieldName ) . SelectExpression
mkProjection <$> case field' of
IR . MCount -> pure countSelect
IR . MExp t -> pure $ textSelect t
IR . MRet returningFields -> mkSelect JASMultipleRows returningFields
let forJson = JsonFor $ ForJson JsonSingleton NoRoot
pure emptySelect { selectFor = forJson , selectProjections = projections }
IR . MOutSinglerowObject singleRowField -> mkSelect JASSingleObject singleRowField
where
mkSelect jsonAggSelect annFields = do
let annSelect = IR . AnnSelectG annFields ( IR . FromIdentifier withAlias ) IR . noTablePermissions IR . noSelectArgs stringifyNum
V . runValidate ( runFromIr $ mkSQLSelect jsonAggSelect annSelect ) ` onLeft ` ( throw500 . tshow )
-- SELECT COUNT(*) FROM [with_alias]
countSelect =
let countProjection = AggregateProjection $ Aliased ( CountAggregate StarCountable ) " count "
in emptySelect
{ selectProjections = [ countProjection ] ,
selectFrom = Just $ TSQL . FromIdentifier withAlias
}
textSelect t =
let textProjection = ExpressionProjection $ Aliased ( ValueExpression ( ODBC . TextValue t ) ) " exp "
in emptySelect { selectProjections = [ textProjection ] }
2021-02-23 20:37:27 +03:00
-- subscription
2021-09-24 01:56:37 +03:00
msDBSubscriptionPlan ::
forall m .
( MonadError QErr m ,
2021-10-22 17:49:15 +03:00
MonadIO m ,
MonadBaseControl IO m
2021-09-24 01:56:37 +03:00
) =>
UserInfo ->
SourceName ->
SourceConfig 'MSSQL ->
InsOrdHashMap G . Name ( QueryDB 'MSSQL ( Const Void ) ( UnpreparedValue 'MSSQL ) ) ->
m ( LiveQueryPlan 'MSSQL ( MultiplexedQuery 'MSSQL ) )
2021-09-23 15:37:56 +03:00
msDBSubscriptionPlan UserInfo { _uiSession , _uiRole } _sourceName sourceConfig rootFields = do
2021-05-21 14:37:34 +03:00
( reselect , prepareState ) <- planSubscription rootFields _uiSession
2021-06-08 06:50:24 +03:00
cohortVariables <- prepareStateCohortVariables sourceConfig _uiSession prepareState
let parameterizedPlan = ParameterizedLiveQueryPlan _uiRole $ MultiplexedQuery' reselect
2021-05-21 14:37:34 +03:00
2021-09-24 01:56:37 +03:00
pure $
LiveQueryPlan parameterizedPlan sourceConfig cohortVariables
2021-05-21 14:37:34 +03:00
2021-10-22 17:49:15 +03:00
prepareStateCohortVariables :: ( MonadError QErr m , MonadIO m , MonadBaseControl IO m ) => SourceConfig 'MSSQL -> SessionVariables -> PrepareState -> m CohortVariables
2021-06-08 06:50:24 +03:00
prepareStateCohortVariables sourceConfig session prepState = do
( namedVars , posVars ) <- validateVariables sourceConfig session prepState
2021-09-24 01:56:37 +03:00
let PrepareState { sessionVariables } = prepState
pure $
mkCohortVariables
sessionVariables
session
namedVars
posVars
2021-06-08 06:50:24 +03:00
-- | Ensure that the set of variables (with value instantiations) that occur in
-- a (RQL) query produce a well-formed and executable (SQL) query when
-- considered in isolation.
--
-- This helps avoiding cascading failures in multiplexed queries.
--
-- c.f. https://github.com/hasura/graphql-engine-mono/issues/1210.
validateVariables ::
2021-10-22 17:49:15 +03:00
( MonadError QErr m , MonadIO m , MonadBaseControl IO m ) =>
2021-06-08 06:50:24 +03:00
SourceConfig 'MSSQL ->
SessionVariables ->
PrepareState ->
m ( ValidatedQueryVariables , ValidatedSyntheticVariables )
validateVariables sourceConfig sessionVariableValues prepState = do
2021-09-24 01:56:37 +03:00
let PrepareState { sessionVariables , namedArguments , positionalArguments } = prepState
-- We generate a single 'canary' query in the form:
--
-- SELECT ... [session].[x-hasura-foo] as [x-hasura-foo], ... as a, ... as b, ...
-- FROM OPENJSON('...')
-- WITH ([x-hasura-foo] NVARCHAR(MAX)) as [session]
--
-- where 'a', 'b', etc. are aliases given to positional arguments.
-- Named arguments and session variables are aliased to themselves.
--
-- The idea being that if the canary query succeeds we can be
-- reasonably confident that adding these variables to a query being
-- polled will not crash the poller.
occSessionVars =
filterSessionVariables
( \ k _ -> Set . member k sessionVariables )
sessionVariableValues
expSes , expNamed , expPos :: [ Aliased Expression ]
expSes = sessionReference <$> getSessionVariables occSessionVars
expNamed =
map
( \ ( n , v ) -> Aliased ( ValueExpression ( RQLColumn . cvValue v ) ) ( G . unName n )
)
$ Map . toList $ namedArguments
-- For positional args we need to be a bit careful not to capture names
-- from expNamed and expSes (however unlikely)
expPos =
zipWith
( \ n v -> Aliased ( ValueExpression ( RQLColumn . cvValue v ) ) n )
( freshVars ( expNamed <> expSes ) )
positionalArguments
projAll :: [ Projection ]
projAll = map ExpressionProjection ( expSes <> expNamed <> expPos )
canaryQuery =
if null projAll
then Nothing
else
Just $
renderQuery
emptySelect
{ selectProjections = projAll ,
selectFrom = sessionOpenJson occSessionVars
}
2021-06-08 06:50:24 +03:00
2021-09-24 01:56:37 +03:00
onJust
canaryQuery
( \ q -> do
_ :: [ [ ODBC . Value ] ] <- withMSSQLPool ( _mscConnectionPool sourceConfig ) ( ` ODBC . query ` q )
pure ()
)
2021-06-08 06:50:24 +03:00
2021-09-24 01:56:37 +03:00
pure
( ValidatedVariables $ txtEncodedColVal <$> namedArguments ,
ValidatedVariables $ txtEncodedColVal <$> positionalArguments
)
where
renderQuery :: Select -> ODBC . Query
renderQuery = toQueryFlat . fromSelect
freshVars :: [ Aliased a ] -> [ Text ]
freshVars boundNames = filter ( not . ( ` elem ` map aliasedAlias boundNames ) ) chars
-- Infinite list of expression aliases.
chars :: [ Text ]
chars = [ y T .<>> x | y <- [ " " ] <|> chars , x <- [ 'a' .. 'z' ] ]
sessionOpenJson :: SessionVariables -> Maybe From
sessionOpenJson occSessionVars =
nonEmpty ( getSessionVariables occSessionVars )
<&> \ fields ->
FromOpenJson $
Aliased
( OpenJson
( ValueExpression $ ODBC . TextValue $ lbsToTxt $ J . encode occSessionVars )
( pure ( sessField <$> fields ) )
)
" session "
sessField :: Text -> JsonFieldSpec
sessField var = StringField var Nothing
sessionReference :: Text -> Aliased Expression
sessionReference var = Aliased ( ColumnExpression ( TSQL . FieldName var " session " ) ) var
2021-09-22 13:43:05 +03:00
--------------------------------------------------------------------------------
-- Remote Relationships (e.g. DB-to-DB Joins, remote schema joins, etc.)
--------------------------------------------------------------------------------
-- | Construct an action (i.e. 'DBStepInfo') which can marshal some remote
-- relationship information into a form that SQL Server can query against.
--
-- XXX: Currently unimplemented; the Postgres implementation uses
-- @jsonb_to_recordset@ to query the remote relationship, however this
-- functionality doesn't exist in SQL Server.
--
-- NOTE: The following typeclass constraints will be necessary when implementing
-- this function for real:
--
-- @
-- MonadQueryTags m
-- Backend 'MSSQL
-- @
2021-09-24 01:56:37 +03:00
msDBRemoteRelationshipPlan ::
forall m .
( MonadError QErr m
) =>
UserInfo ->
SourceName ->
SourceConfig 'MSSQL ->
2021-09-22 13:43:05 +03:00
-- | List of json objects, each of which becomes a row of the table.
2021-09-24 01:56:37 +03:00
NonEmpty J . Object ->
2021-09-22 13:43:05 +03:00
-- | The above objects have this schema
--
-- XXX: What is this for/what does this mean?
2021-09-24 01:56:37 +03:00
HashMap RQLTypes . FieldName ( RQLTypes . Column 'MSSQL , RQLTypes . ScalarType 'MSSQL ) ->
2021-09-22 13:43:05 +03:00
-- | This is a field name from the lhs that *has* to be selected in the
-- response along with the relationship.
2021-09-24 01:56:37 +03:00
RQLTypes . FieldName ->
( RQLTypes . FieldName , SourceRelationshipSelection 'MSSQL ( Const Void ) UnpreparedValue ) ->
m ( DBStepInfo 'MSSQL )
2021-09-22 13:43:05 +03:00
msDBRemoteRelationshipPlan _userInfo _sourceName _sourceConfig _lhs _lhsSchema _argumentId _relationship = do
throw500 " mkDBRemoteRelationshipPlan: SQL Server (MSSQL) does not currently support generalized joins. "