{-# OPTIONS_GHC -fno-warn-orphans #-}

module Hasura.Backends.BigQuery.Instances.Execute () where

import Data.Aeson qualified as Aeson
import Data.Aeson.Text qualified as Aeson
import Data.Environment qualified as Env
import Data.HashMap.Strict qualified as Map
import Data.HashMap.Strict.InsOrd qualified as OMap
import Data.Text qualified as T
import Data.Text.Lazy qualified as LT
import Data.Text.Lazy.Builder qualified as LT
import Data.Vector qualified as V
import Hasura.Backends.BigQuery.Execute qualified as DataLoader
import Hasura.Backends.BigQuery.FromIr qualified as BigQuery
import Hasura.Backends.BigQuery.Plan
import Hasura.Backends.BigQuery.ToQuery qualified as ToQuery
import Hasura.Backends.BigQuery.Types qualified as BigQuery
import Hasura.Base.Error
import Hasura.Base.Error qualified as E
import Hasura.EncJSON
import Hasura.GraphQL.Execute.Backend
import Hasura.GraphQL.Namespace (RootFieldAlias)
import Hasura.GraphQL.Schema.Options qualified as Options
import Hasura.Prelude
import Hasura.QueryTags
  ( emptyQueryTagsComment,
  )
import Hasura.RQL.IR
import Hasura.RQL.IR.Select qualified as IR
import Hasura.RQL.Types.Backend
import Hasura.RQL.Types.Column
import Hasura.RQL.Types.Common
import Hasura.RQL.Types.Function
import Hasura.SQL.AnyBackend qualified as AB
import Hasura.SQL.Backend
import Hasura.Session
import Hasura.Tracing qualified as Tracing

instance BackendExecute 'BigQuery where
  type PreparedQuery 'BigQuery = Text
  type MultiplexedQuery 'BigQuery = Void
  type ExecutionMonad 'BigQuery = Tracing.TraceT (ExceptT QErr IO)

  mkDBQueryPlan :: UserInfo
-> Environment
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
mkDBQueryPlan = UserInfo
-> Environment
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
forall (m :: * -> *).
MonadError QErr m =>
UserInfo
-> Environment
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
bqDBQueryPlan
  mkDBMutationPlan :: UserInfo
-> StringifyNumbers
-> SourceName
-> SourceConfig 'BigQuery
-> MutationDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
mkDBMutationPlan = UserInfo
-> StringifyNumbers
-> SourceName
-> SourceConfig 'BigQuery
-> MutationDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
forall (m :: * -> *).
MonadError QErr m =>
UserInfo
-> StringifyNumbers
-> SourceName
-> SourceConfig 'BigQuery
-> MutationDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
bqDBMutationPlan
  mkLiveQuerySubscriptionPlan :: UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> Maybe Name
-> RootFieldMap
     (QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery))
-> m (SubscriptionQueryPlan 'BigQuery (MultiplexedQuery 'BigQuery))
mkLiveQuerySubscriptionPlan UserInfo
_ SourceName
_ SourceConfig 'BigQuery
_ Maybe Name
_ RootFieldMap (QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery))
_ =
    Text -> m (SubscriptionQueryPlan 'BigQuery Void)
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 Text
"Cannot currently perform subscriptions on BigQuery sources."
  mkDBStreamingSubscriptionPlan :: UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> (RootFieldAlias,
    QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery))
-> m (SubscriptionQueryPlan 'BigQuery (MultiplexedQuery 'BigQuery))
mkDBStreamingSubscriptionPlan UserInfo
_ SourceName
_ SourceConfig 'BigQuery
_ (RootFieldAlias,
 QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery))
_ =
    Text -> m (SubscriptionQueryPlan 'BigQuery Void)
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 Text
"Cannot currently perform subscriptions on BigQuery sources."
  mkDBQueryExplain :: RootFieldAlias
-> UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (AnyBackend DBStepInfo)
mkDBQueryExplain = RootFieldAlias
-> UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (AnyBackend DBStepInfo)
forall (m :: * -> *).
MonadError QErr m =>
RootFieldAlias
-> UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (AnyBackend DBStepInfo)
bqDBQueryExplain
  mkSubscriptionExplain :: SubscriptionQueryPlan 'BigQuery (MultiplexedQuery 'BigQuery)
-> m SubscriptionQueryPlanExplanation
mkSubscriptionExplain SubscriptionQueryPlan 'BigQuery (MultiplexedQuery 'BigQuery)
_ =
    Text -> m SubscriptionQueryPlanExplanation
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 Text
"Cannot currently retrieve query execution plans on BigQuery sources."

  -- NOTE: Currently unimplemented!.
  --
  -- This function is just a stub for future implementation; for now it just
  -- throws a 500 error.
  mkDBRemoteRelationshipPlan :: UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> NonEmpty Object
-> HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery)
-> FieldName
-> (FieldName,
    SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
-> m (DBStepInfo 'BigQuery)
mkDBRemoteRelationshipPlan =
    UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> NonEmpty Object
-> HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery)
-> FieldName
-> (FieldName,
    SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
-> m (DBStepInfo 'BigQuery)
forall (m :: * -> *).
MonadError QErr m =>
UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> NonEmpty Object
-> HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery)
-> FieldName
-> (FieldName,
    SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
-> m (DBStepInfo 'BigQuery)
bqDBRemoteRelationshipPlan

-- query

bqDBQueryPlan ::
  forall m.
  ( MonadError E.QErr m
  ) =>
  UserInfo ->
  Env.Environment ->
  SourceName ->
  SourceConfig 'BigQuery ->
  QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery) ->
  m (DBStepInfo 'BigQuery)
bqDBQueryPlan :: UserInfo
-> Environment
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
bqDBQueryPlan UserInfo
userInfo Environment
_env SourceName
sourceName SourceConfig 'BigQuery
sourceConfig QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
qrf = do
  -- TODO (naveen): Append query tags to the query
  Select
select <- FromIrConfig
-> UserInfo
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m Select
forall (m :: * -> *).
MonadError QErr m =>
FromIrConfig
-> UserInfo
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m Select
planNoPlan (BigQuerySourceConfig -> FromIrConfig
BigQuery.bigQuerySourceConfigToFromIrConfig SourceConfig 'BigQuery
BigQuerySourceConfig
sourceConfig) UserInfo
userInfo QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
qrf
  let action :: TraceT (ExceptT QErr IO) EncJSON
action = do
        Either ExecuteProblem RecordSet
result <-
          BigQuerySourceConfig
-> Execute RecordSet
-> TraceT (ExceptT QErr IO) (Either ExecuteProblem RecordSet)
forall (m :: * -> *).
MonadIO m =>
BigQuerySourceConfig
-> Execute RecordSet -> m (Either ExecuteProblem RecordSet)
DataLoader.runExecute
            SourceConfig 'BigQuery
BigQuerySourceConfig
sourceConfig
            (Select -> Execute RecordSet
DataLoader.executeSelect Select
select)
        case Either ExecuteProblem RecordSet
result of
          Left ExecuteProblem
err -> Text -> Value -> TraceT (ExceptT QErr IO) EncJSON
forall (m :: * -> *) a. QErrM m => Text -> Value -> m a
throw500WithDetail (ShowDetails -> ExecuteProblem -> Text
DataLoader.executeProblemMessage ShowDetails
DataLoader.HideDetails ExecuteProblem
err) (Value -> TraceT (ExceptT QErr IO) EncJSON)
-> Value -> TraceT (ExceptT QErr IO) EncJSON
forall a b. (a -> b) -> a -> b
$ ExecuteProblem -> Value
forall a. ToJSON a => a -> Value
Aeson.toJSON ExecuteProblem
err
          Right RecordSet
recordSet -> EncJSON -> TraceT (ExceptT QErr IO) EncJSON
forall (f :: * -> *) a. Applicative f => a -> f a
pure (EncJSON -> TraceT (ExceptT QErr IO) EncJSON)
-> EncJSON -> TraceT (ExceptT QErr IO) EncJSON
forall a b. (a -> b) -> a -> b
$! Cardinality -> RecordSet -> EncJSON
recordSetToEncJSON (Select -> Cardinality
BigQuery.selectCardinality Select
select) RecordSet
recordSet
  DBStepInfo 'BigQuery -> m (DBStepInfo 'BigQuery)
forall (f :: * -> *) a. Applicative f => a -> f a
pure (DBStepInfo 'BigQuery -> m (DBStepInfo 'BigQuery))
-> DBStepInfo 'BigQuery -> m (DBStepInfo 'BigQuery)
forall a b. (a -> b) -> a -> b
$ SourceName
-> SourceConfig 'BigQuery
-> Maybe (PreparedQuery 'BigQuery)
-> ExecutionMonad 'BigQuery EncJSON
-> DBStepInfo 'BigQuery
forall (b :: BackendType).
SourceName
-> SourceConfig b
-> Maybe (PreparedQuery b)
-> ExecutionMonad b EncJSON
-> DBStepInfo b
DBStepInfo @'BigQuery SourceName
sourceName SourceConfig 'BigQuery
sourceConfig (Text -> Maybe Text
forall a. a -> Maybe a
Just (Select -> Text
selectSQLTextForExplain Select
select)) TraceT (ExceptT QErr IO) EncJSON
ExecutionMonad 'BigQuery EncJSON
action

-- | Convert the dataloader's 'RecordSet' type to JSON.
recordSetToEncJSON :: BigQuery.Cardinality -> DataLoader.RecordSet -> EncJSON
recordSetToEncJSON :: Cardinality -> RecordSet -> EncJSON
recordSetToEncJSON Cardinality
cardinality DataLoader.RecordSet {Vector (InsOrdHashMap FieldNameText OutputValue)
$sel:rows:RecordSet :: RecordSet -> Vector (InsOrdHashMap FieldNameText OutputValue)
rows :: Vector (InsOrdHashMap FieldNameText OutputValue)
rows} =
  case Cardinality
cardinality of
    Cardinality
BigQuery.One
      | Just InsOrdHashMap FieldNameText OutputValue
row <- Vector (InsOrdHashMap FieldNameText OutputValue)
rows Vector (InsOrdHashMap FieldNameText OutputValue)
-> Int -> Maybe (InsOrdHashMap FieldNameText OutputValue)
forall a. Vector a -> Int -> Maybe a
V.!? Int
0 -> InsOrdHashMap FieldNameText OutputValue -> EncJSON
encJFromRecord InsOrdHashMap FieldNameText OutputValue
row
      | Bool
otherwise -> [EncJSON] -> EncJSON
encJFromList (Vector EncJSON -> [EncJSON]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList ((InsOrdHashMap FieldNameText OutputValue -> EncJSON)
-> Vector (InsOrdHashMap FieldNameText OutputValue)
-> Vector EncJSON
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap InsOrdHashMap FieldNameText OutputValue -> EncJSON
encJFromRecord Vector (InsOrdHashMap FieldNameText OutputValue)
rows))
    Cardinality
BigQuery.Many -> [EncJSON] -> EncJSON
encJFromList (Vector EncJSON -> [EncJSON]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList ((InsOrdHashMap FieldNameText OutputValue -> EncJSON)
-> Vector (InsOrdHashMap FieldNameText OutputValue)
-> Vector EncJSON
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap InsOrdHashMap FieldNameText OutputValue -> EncJSON
encJFromRecord Vector (InsOrdHashMap FieldNameText OutputValue)
rows))
  where
    encJFromRecord :: InsOrdHashMap FieldNameText OutputValue -> EncJSON
encJFromRecord =
      InsOrdHashMap Text EncJSON -> EncJSON
encJFromInsOrdHashMap (InsOrdHashMap Text EncJSON -> EncJSON)
-> (InsOrdHashMap FieldNameText OutputValue
    -> InsOrdHashMap Text EncJSON)
-> InsOrdHashMap FieldNameText OutputValue
-> EncJSON
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (OutputValue -> EncJSON)
-> InsOrdHashMap Text OutputValue -> InsOrdHashMap Text EncJSON
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap OutputValue -> EncJSON
encJFromOutputValue (InsOrdHashMap Text OutputValue -> InsOrdHashMap Text EncJSON)
-> (InsOrdHashMap FieldNameText OutputValue
    -> InsOrdHashMap Text OutputValue)
-> InsOrdHashMap FieldNameText OutputValue
-> InsOrdHashMap Text EncJSON
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (FieldNameText -> Text)
-> InsOrdHashMap FieldNameText OutputValue
-> InsOrdHashMap Text OutputValue
forall k' k v.
(Eq k', Hashable k') =>
(k -> k') -> InsOrdHashMap k v -> InsOrdHashMap k' v
OMap.mapKeys FieldNameText -> Text
coerce
    encJFromOutputValue :: OutputValue -> EncJSON
encJFromOutputValue OutputValue
outputValue =
      case OutputValue
outputValue of
        OutputValue
DataLoader.NullOutputValue -> Value -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Value
Aeson.Null
        DataLoader.DecimalOutputValue Decimal
i -> Decimal -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Decimal
i
        DataLoader.BigDecimalOutputValue BigDecimal
i -> BigDecimal -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue BigDecimal
i
        DataLoader.FloatOutputValue Float64
i -> Float64 -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Float64
i
        DataLoader.TextOutputValue Text
i -> Text -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Text
i
        DataLoader.BytesOutputValue Base64
i -> Base64 -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Base64
i
        DataLoader.DateOutputValue Date
i -> Date -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Date
i
        DataLoader.TimestampOutputValue Timestamp
i -> Timestamp -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Timestamp
i
        DataLoader.TimeOutputValue Time
i -> Time -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Time
i
        DataLoader.DatetimeOutputValue Datetime
i -> Datetime -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Datetime
i
        DataLoader.GeographyOutputValue Geography
i -> Geography -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Geography
i
        DataLoader.BoolOutputValue Bool
i -> Bool -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Bool
i
        DataLoader.IntegerOutputValue Int64
i -> Int64 -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue Int64
i
        DataLoader.ArrayOutputValue Vector OutputValue
vector ->
          [EncJSON] -> EncJSON
encJFromList (Vector EncJSON -> [EncJSON]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList ((OutputValue -> EncJSON) -> Vector OutputValue -> Vector EncJSON
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap OutputValue -> EncJSON
encJFromOutputValue Vector OutputValue
vector))
        -- Really, the case below shouldn't be happening. But if it
        -- does, it's not a problem either. The output will just have
        -- a record in it.
        DataLoader.RecordOutputValue InsOrdHashMap FieldNameText OutputValue
record -> InsOrdHashMap FieldNameText OutputValue -> EncJSON
encJFromRecord InsOrdHashMap FieldNameText OutputValue
record

-- mutation

bqDBMutationPlan ::
  forall m.
  ( MonadError E.QErr m
  ) =>
  UserInfo ->
  Options.StringifyNumbers ->
  SourceName ->
  SourceConfig 'BigQuery ->
  MutationDB 'BigQuery Void (UnpreparedValue 'BigQuery) ->
  m (DBStepInfo 'BigQuery)
bqDBMutationPlan :: UserInfo
-> StringifyNumbers
-> SourceName
-> SourceConfig 'BigQuery
-> MutationDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
bqDBMutationPlan UserInfo
_userInfo StringifyNumbers
_stringifyNum SourceName
_sourceName SourceConfig 'BigQuery
_sourceConfig MutationDB 'BigQuery Void (UnpreparedValue 'BigQuery)
_mrf =
  Text -> m (DBStepInfo 'BigQuery)
forall (m :: * -> *) a. QErrM m => Text -> m a
throw500 Text
"mutations are not supported in BigQuery; this should be unreachable"

-- explain

bqDBQueryExplain ::
  MonadError E.QErr m =>
  RootFieldAlias ->
  UserInfo ->
  SourceName ->
  SourceConfig 'BigQuery ->
  QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery) ->
  m (AB.AnyBackend DBStepInfo)
bqDBQueryExplain :: RootFieldAlias
-> UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (AnyBackend DBStepInfo)
bqDBQueryExplain RootFieldAlias
fieldName UserInfo
userInfo SourceName
sourceName SourceConfig 'BigQuery
sourceConfig QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
qrf = do
  Select
select <- FromIrConfig
-> UserInfo
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m Select
forall (m :: * -> *).
MonadError QErr m =>
FromIrConfig
-> UserInfo
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m Select
planNoPlan (BigQuerySourceConfig -> FromIrConfig
BigQuery.bigQuerySourceConfigToFromIrConfig SourceConfig 'BigQuery
BigQuerySourceConfig
sourceConfig) UserInfo
userInfo QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
qrf
  let textSQL :: Text
textSQL = Select -> Text
selectSQLTextForExplain Select
select
  AnyBackend DBStepInfo -> m (AnyBackend DBStepInfo)
forall (f :: * -> *) a. Applicative f => a -> f a
pure (AnyBackend DBStepInfo -> m (AnyBackend DBStepInfo))
-> AnyBackend DBStepInfo -> m (AnyBackend DBStepInfo)
forall a b. (a -> b) -> a -> b
$
    DBStepInfo 'BigQuery -> AnyBackend DBStepInfo
forall (b :: BackendType) (i :: BackendType -> *).
HasTag b =>
i b -> AnyBackend i
AB.mkAnyBackend (DBStepInfo 'BigQuery -> AnyBackend DBStepInfo)
-> DBStepInfo 'BigQuery -> AnyBackend DBStepInfo
forall a b. (a -> b) -> a -> b
$
      SourceName
-> SourceConfig 'BigQuery
-> Maybe (PreparedQuery 'BigQuery)
-> ExecutionMonad 'BigQuery EncJSON
-> DBStepInfo 'BigQuery
forall (b :: BackendType).
SourceName
-> SourceConfig b
-> Maybe (PreparedQuery b)
-> ExecutionMonad b EncJSON
-> DBStepInfo b
DBStepInfo @'BigQuery SourceName
sourceName SourceConfig 'BigQuery
sourceConfig Maybe (PreparedQuery 'BigQuery)
forall a. Maybe a
Nothing (ExecutionMonad 'BigQuery EncJSON -> DBStepInfo 'BigQuery)
-> ExecutionMonad 'BigQuery EncJSON -> DBStepInfo 'BigQuery
forall a b. (a -> b) -> a -> b
$
        EncJSON -> TraceT (ExceptT QErr IO) EncJSON
forall (f :: * -> *) a. Applicative f => a -> f a
pure (EncJSON -> TraceT (ExceptT QErr IO) EncJSON)
-> EncJSON -> TraceT (ExceptT QErr IO) EncJSON
forall a b. (a -> b) -> a -> b
$
          ExplainPlan -> EncJSON
forall a. ToJSON a => a -> EncJSON
encJFromJValue (ExplainPlan -> EncJSON) -> ExplainPlan -> EncJSON
forall a b. (a -> b) -> a -> b
$
            RootFieldAlias -> Maybe Text -> Maybe [Text] -> ExplainPlan
ExplainPlan
              RootFieldAlias
fieldName
              (Text -> Maybe Text
forall a. a -> Maybe a
Just (Text -> Maybe Text) -> Text -> Maybe Text
forall a b. (a -> b) -> a -> b
$ Text
textSQL)
              ([Text] -> Maybe [Text]
forall a. a -> Maybe a
Just ([Text] -> Maybe [Text]) -> [Text] -> Maybe [Text]
forall a b. (a -> b) -> a -> b
$ Text -> [Text]
T.lines (Text -> [Text]) -> Text -> [Text]
forall a b. (a -> b) -> a -> b
$ Text
textSQL)

-- | Get the SQL text for a select, with parameters left as $1, $2, .. holes.
selectSQLTextForExplain :: BigQuery.Select -> Text
selectSQLTextForExplain :: Select -> Text
selectSQLTextForExplain =
  Text -> Text
LT.toStrict
    (Text -> Text) -> (Select -> Text) -> Select -> Text
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Builder -> Text
LT.toLazyText
    (Builder -> Text) -> (Select -> Builder) -> Select -> Text
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (Builder, InsOrdHashMap Int Value) -> Builder
forall a b. (a, b) -> a
fst
    ((Builder, InsOrdHashMap Int Value) -> Builder)
-> (Select -> (Builder, InsOrdHashMap Int Value))
-> Select
-> Builder
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Printer -> (Builder, InsOrdHashMap Int Value)
ToQuery.renderBuilderPretty
    (Printer -> (Builder, InsOrdHashMap Int Value))
-> (Select -> Printer)
-> Select
-> (Builder, InsOrdHashMap Int Value)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Select -> Printer
ToQuery.fromSelect

--------------------------------------------------------------------------------
-- 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 BigQuery 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 BigQuery.
--
-- NOTE: The following typeclass constraints will be necessary when implementing
-- this function for real:
--
-- @
--   MonadQueryTags m
--   Backend 'BigQuery
-- @
bqDBRemoteRelationshipPlan ::
  forall m.
  ( MonadError QErr m
  ) =>
  UserInfo ->
  SourceName ->
  SourceConfig 'BigQuery ->
  -- | List of json objects, each of which becomes a row of the table.
  NonEmpty Aeson.Object ->
  -- | The above objects have this schema
  --
  -- XXX: What is this for/what does this mean?
  HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery) ->
  -- | This is a field name from the lhs that *has* to be selected in the
  -- response along with the relationship.
  FieldName ->
  (FieldName, SourceRelationshipSelection 'BigQuery Void UnpreparedValue) ->
  m (DBStepInfo 'BigQuery)
bqDBRemoteRelationshipPlan :: UserInfo
-> SourceName
-> SourceConfig 'BigQuery
-> NonEmpty Object
-> HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery)
-> FieldName
-> (FieldName,
    SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
-> m (DBStepInfo 'BigQuery)
bqDBRemoteRelationshipPlan UserInfo
userInfo SourceName
sourceName SourceConfig 'BigQuery
sourceConfig NonEmpty Object
lhs HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery)
lhsSchema FieldName
argumentId (FieldName,
 SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
relationship = do
  (ReaderT QueryTagsComment m (DBStepInfo 'BigQuery)
 -> QueryTagsComment -> m (DBStepInfo 'BigQuery))
-> QueryTagsComment
-> ReaderT QueryTagsComment m (DBStepInfo 'BigQuery)
-> m (DBStepInfo 'BigQuery)
forall a b c. (a -> b -> c) -> b -> a -> c
flip ReaderT QueryTagsComment m (DBStepInfo 'BigQuery)
-> QueryTagsComment -> m (DBStepInfo 'BigQuery)
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT QueryTagsComment
emptyQueryTagsComment (ReaderT QueryTagsComment m (DBStepInfo 'BigQuery)
 -> m (DBStepInfo 'BigQuery))
-> ReaderT QueryTagsComment m (DBStepInfo 'BigQuery)
-> m (DBStepInfo 'BigQuery)
forall a b. (a -> b) -> a -> b
$ UserInfo
-> Environment
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> ReaderT QueryTagsComment m (DBStepInfo 'BigQuery)
forall (m :: * -> *).
MonadError QErr m =>
UserInfo
-> Environment
-> SourceName
-> SourceConfig 'BigQuery
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
-> m (DBStepInfo 'BigQuery)
bqDBQueryPlan UserInfo
userInfo Environment
Env.emptyEnvironment SourceName
sourceName SourceConfig 'BigQuery
sourceConfig QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
rootSelection
  where
    coerceToColumn :: FieldName -> ColumnName
coerceToColumn = Text -> ColumnName
BigQuery.ColumnName (Text -> ColumnName)
-> (FieldName -> Text) -> FieldName -> ColumnName
forall b c a. (b -> c) -> (a -> b) -> a -> c
. FieldName -> Text
getFieldNameTxt
    joinColumnMapping :: HashMap ColumnName (ColumnName, ScalarType)
joinColumnMapping = (FieldName -> ColumnName)
-> HashMap FieldName (ColumnName, ScalarType)
-> HashMap ColumnName (ColumnName, ScalarType)
forall k2 k1 v.
(Eq k2, Hashable k2) =>
(k1 -> k2) -> HashMap k1 v -> HashMap k2 v
mapKeys FieldName -> ColumnName
coerceToColumn HashMap FieldName (Column 'BigQuery, ScalarType 'BigQuery)
HashMap FieldName (ColumnName, ScalarType)
lhsSchema

    rowsArgument :: UnpreparedValue 'BigQuery
    rowsArgument :: UnpreparedValue 'BigQuery
rowsArgument =
      Maybe VariableInfo
-> ColumnValue 'BigQuery -> UnpreparedValue 'BigQuery
forall (b :: BackendType).
Maybe VariableInfo -> ColumnValue b -> UnpreparedValue b
UVParameter Maybe VariableInfo
forall a. Maybe a
Nothing (ColumnValue 'BigQuery -> UnpreparedValue 'BigQuery)
-> ColumnValue 'BigQuery -> UnpreparedValue 'BigQuery
forall a b. (a -> b) -> a -> b
$
        ColumnType 'BigQuery
-> ScalarValue 'BigQuery -> ColumnValue 'BigQuery
forall (b :: BackendType).
ColumnType b -> ScalarValue b -> ColumnValue b
ColumnValue (ScalarType 'BigQuery -> ColumnType 'BigQuery
forall (b :: BackendType). ScalarType b -> ColumnType b
ColumnScalar ScalarType 'BigQuery
ScalarType
BigQuery.StringScalarType) (ScalarValue 'BigQuery -> ColumnValue 'BigQuery)
-> ScalarValue 'BigQuery -> ColumnValue 'BigQuery
forall a b. (a -> b) -> a -> b
$
          Text -> Value
BigQuery.StringValue (Text -> Value) -> (Text -> Text) -> Text -> Value
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Text -> Text
LT.toStrict (Text -> Value) -> Text -> Value
forall a b. (a -> b) -> a -> b
$ NonEmpty Object -> Text
forall a. ToJSON a => a -> Text
Aeson.encodeToLazyText NonEmpty Object
lhs

    recordSetDefinitionList :: [(ColumnName, ScalarType)]
recordSetDefinitionList =
      (FieldName -> ColumnName
coerceToColumn FieldName
argumentId, ScalarType
BigQuery.IntegerScalarType) (ColumnName, ScalarType)
-> [(ColumnName, ScalarType)] -> [(ColumnName, ScalarType)]
forall a. a -> [a] -> [a]
: HashMap ColumnName ScalarType -> [(ColumnName, ScalarType)]
forall k v. HashMap k v -> [(k, v)]
Map.toList (((ColumnName, ScalarType) -> ScalarType)
-> HashMap ColumnName (ColumnName, ScalarType)
-> HashMap ColumnName ScalarType
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (ColumnName, ScalarType) -> ScalarType
forall a b. (a, b) -> b
snd HashMap ColumnName (ColumnName, ScalarType)
joinColumnMapping)

    jsonToRecordSet :: IR.SelectFromG ('BigQuery) (UnpreparedValue 'BigQuery)
    jsonToRecordSet :: SelectFromG 'BigQuery (UnpreparedValue 'BigQuery)
jsonToRecordSet =
      FunctionName 'BigQuery
-> FunctionArgsExp 'BigQuery (UnpreparedValue 'BigQuery)
-> Maybe [(Column 'BigQuery, ScalarType 'BigQuery)]
-> SelectFromG 'BigQuery (UnpreparedValue 'BigQuery)
forall (b :: BackendType) v.
FunctionName b
-> FunctionArgsExp b v
-> Maybe [(Column b, ScalarType b)]
-> SelectFromG b v
IR.FromFunction
        (Text -> Maybe Text -> FunctionName
BigQuery.FunctionName Text
"unnest" Maybe Text
forall a. Maybe a
Nothing)
        ( [ArgumentExp (UnpreparedValue 'BigQuery)]
-> HashMap Text (ArgumentExp (UnpreparedValue 'BigQuery))
-> FunctionArgsExpG (ArgumentExp (UnpreparedValue 'BigQuery))
forall a. [a] -> HashMap Text a -> FunctionArgsExpG a
FunctionArgsExp
            [UnpreparedValue 'BigQuery
-> ArgumentExp (UnpreparedValue 'BigQuery)
forall v. v -> ArgumentExp v
BigQuery.AEInput UnpreparedValue 'BigQuery
rowsArgument]
            HashMap Text (ArgumentExp (UnpreparedValue 'BigQuery))
forall a. Monoid a => a
mempty
        )
        ([(ColumnName, ScalarType)] -> Maybe [(ColumnName, ScalarType)]
forall a. a -> Maybe a
Just [(ColumnName, ScalarType)]
recordSetDefinitionList)

    rootSelection :: QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
rootSelection =
      HashMap (Column 'BigQuery) (Column 'BigQuery)
-> SelectFromG 'BigQuery (UnpreparedValue 'BigQuery)
-> Column 'BigQuery
-> ColumnType 'BigQuery
-> (FieldName,
    SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
-> QueryDB 'BigQuery Void (UnpreparedValue 'BigQuery)
forall (b :: BackendType).
Backend b =>
HashMap (Column b) (Column b)
-> SelectFromG b (UnpreparedValue b)
-> Column b
-> ColumnType b
-> (FieldName, SourceRelationshipSelection b Void UnpreparedValue)
-> QueryDB b Void (UnpreparedValue b)
convertRemoteSourceRelationship
        ((ColumnName, ScalarType) -> ColumnName
forall a b. (a, b) -> a
fst ((ColumnName, ScalarType) -> ColumnName)
-> HashMap ColumnName (ColumnName, ScalarType)
-> HashMap ColumnName ColumnName
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> HashMap ColumnName (ColumnName, ScalarType)
joinColumnMapping)
        SelectFromG 'BigQuery (UnpreparedValue 'BigQuery)
jsonToRecordSet
        (Text -> ColumnName
BigQuery.ColumnName (Text -> ColumnName) -> Text -> ColumnName
forall a b. (a -> b) -> a -> b
$ FieldName -> Text
getFieldNameTxt FieldName
argumentId)
        (ScalarType 'BigQuery -> ColumnType 'BigQuery
forall (b :: BackendType). ScalarType b -> ColumnType b
ColumnScalar ScalarType 'BigQuery
ScalarType
BigQuery.IntegerScalarType)
        (FieldName,
 SourceRelationshipSelection 'BigQuery Void UnpreparedValue)
relationship