-- | Translate from the DML to the BigQuery dialect.
module Hasura.Backends.BigQuery.FromIr
  ( mkSQLSelect,
    fromRootField,
    fromSelectAggregate,
    Error (..),
    runFromIr,
    FromIr,
    FromIrWriter (..),
    FromIrConfig (..),
    defaultFromIrConfig,
    bigQuerySourceConfigToFromIrConfig,
    Top (..), -- Re-export for FromIrConfig.
  )
where

import Control.Applicative (getConst)
import Control.Monad.Validate
import Data.HashMap.Strict qualified as HashMap
import Data.Int qualified as Int
import Data.List.Extended (appendToNonEmpty)
import Data.List.NonEmpty qualified as NE
import Data.Map.Strict (Map)
import Data.Map.Strict qualified as M
import Data.Text qualified as T
import Hasura.Backends.BigQuery.Instances.Types ()
import Hasura.Backends.BigQuery.Source (BigQuerySourceConfig (..))
import Hasura.Backends.BigQuery.Types as BigQuery
import Hasura.Function.Cache qualified as Functions
import Hasura.NativeQuery.IR (NativeQuery (..))
import Hasura.NativeQuery.Metadata (InterpolatedQuery)
import Hasura.NativeQuery.Types (NativeQueryName (..))
import Hasura.Prelude
import Hasura.RQL.IR qualified as Ir
import Hasura.RQL.Types.BackendType
import Hasura.RQL.Types.Column qualified as Rql
import Hasura.RQL.Types.Common qualified as Rql
import Hasura.RQL.Types.Relationships.Local (Nullable (..))
import Hasura.RQL.Types.Relationships.Local qualified as Rql
import Language.GraphQL.Draft.Syntax qualified as G

--------------------------------------------------------------------------------
-- Types

-- | Most of these errors should be checked for legitimacy.
data Error
  = FromTypeUnsupported (Ir.SelectFromG 'BigQuery Expression)
  | NoOrderSpecifiedInOrderBy
  | MalformedAgg
  | FieldTypeUnsupportedForNow (Ir.AnnFieldG 'BigQuery Void Expression)
  | AggTypeUnsupportedForNow (Ir.TableAggregateFieldG 'BigQuery Void Expression)
  | NodesUnsupportedForNow (Ir.TableAggregateFieldG 'BigQuery Void Expression)
  | NoProjectionFields
  | NoAggregatesMustBeABug
  | UnsupportedArraySelect (Ir.ArraySelectG 'BigQuery Void Expression)
  | UnsupportedOpExpG (Ir.OpExpG 'BigQuery Expression)
  | UnsupportedSQLExp Expression
  | UnsupportedDistinctOn
  | UnexpectedEmptyList
  | InvalidIntegerishSql Expression
  | ConnectionsNotSupported
  | ActionsNotSupported
  | -- | https://github.com/hasura/graphql-engine/issues/8526
    ComputedFieldsBooleanExpressionNotSupported
  | -- | https://github.com/hasura/graphql-engine/issues/8526
    ComputedFieldsOrderByNotSupported
  | -- | https://github.com/hasura/graphql-engine/issues/8521
    ScalarComputedFieldsNotSupported
  | NoParentEntityInternalError

instance Show Error where
  show :: Error -> [Char]
show =
    \case
      FromTypeUnsupported {} -> [Char]
"FromTypeUnsupported"
      NoOrderSpecifiedInOrderBy {} -> [Char]
"NoOrderSpecifiedInOrderBy"
      MalformedAgg {} -> [Char]
"MalformedAgg"
      FieldTypeUnsupportedForNow {} -> [Char]
"FieldTypeUnsupportedForNow"
      AggTypeUnsupportedForNow {} -> [Char]
"AggTypeUnsupportedForNow"
      NodesUnsupportedForNow {} -> [Char]
"NodesUnsupportedForNow"
      NoProjectionFields {} -> [Char]
"NoProjectionFields"
      NoAggregatesMustBeABug {} -> [Char]
"NoAggregatesMustBeABug"
      UnsupportedArraySelect {} -> [Char]
"UnsupportedArraySelect"
      UnsupportedOpExpG {} -> [Char]
"UnsupportedOpExpG"
      UnsupportedSQLExp {} -> [Char]
"UnsupportedSQLExp"
      UnsupportedDistinctOn {} -> [Char]
"UnsupportedDistinctOn"
      UnexpectedEmptyList {} -> [Char]
"UnexpectedEmptyList"
      InvalidIntegerishSql {} -> [Char]
"InvalidIntegerishSql"
      ConnectionsNotSupported {} -> [Char]
"ConnectionsNotSupported"
      ActionsNotSupported {} -> [Char]
"ActionsNotSupported"
      ComputedFieldsBooleanExpressionNotSupported {} -> [Char]
"ComputedFieldsBooleanExpressionNotSupported"
      ComputedFieldsOrderByNotSupported {} -> [Char]
"ComputedFieldsOrderByNotSupported"
      ScalarComputedFieldsNotSupported {} -> [Char]
"ScalarComputedFieldsNotSupported"
      NoParentEntityInternalError {} -> [Char]
"NoParentEntityInternalError"

-- | The base monad used throughout this module for all conversion
-- functions.
--
-- It's a Validate, so it'll continue going when it encounters errors
-- to accumulate as many as possible.
--
-- It also contains a mapping from entity prefixes to counters. So if
-- my prefix is "table" then there'll be a counter that lets me
-- generate table1, table2, etc. Same for any other prefix needed
-- (e.g. names for joins).
--
-- A ReaderT is used around this in most of the module too, for
-- setting the current entity that a given field name refers to. See
-- @fromColumn@.
newtype FromIr a = FromIr
  { forall a.
FromIr a
-> ReaderT
     FromIrReader
     (StateT
        FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
     a
unFromIr ::
      ReaderT
        FromIrReader
        ( StateT
            FromIrState
            ( WriterT
                FromIrWriter
                ( Validate (NonEmpty Error)
                )
            )
        )
        a
  }
  deriving ((forall a b. (a -> b) -> FromIr a -> FromIr b)
-> (forall a b. a -> FromIr b -> FromIr a) -> Functor FromIr
forall a b. a -> FromIr b -> FromIr a
forall a b. (a -> b) -> FromIr a -> FromIr b
forall (f :: * -> *).
(forall a b. (a -> b) -> f a -> f b)
-> (forall a b. a -> f b -> f a) -> Functor f
$cfmap :: forall a b. (a -> b) -> FromIr a -> FromIr b
fmap :: forall a b. (a -> b) -> FromIr a -> FromIr b
$c<$ :: forall a b. a -> FromIr b -> FromIr a
<$ :: forall a b. a -> FromIr b -> FromIr a
Functor, Functor FromIr
Functor FromIr
-> (forall a. a -> FromIr a)
-> (forall a b. FromIr (a -> b) -> FromIr a -> FromIr b)
-> (forall a b c.
    (a -> b -> c) -> FromIr a -> FromIr b -> FromIr c)
-> (forall a b. FromIr a -> FromIr b -> FromIr b)
-> (forall a b. FromIr a -> FromIr b -> FromIr a)
-> Applicative FromIr
forall a. a -> FromIr a
forall a b. FromIr a -> FromIr b -> FromIr a
forall a b. FromIr a -> FromIr b -> FromIr b
forall a b. FromIr (a -> b) -> FromIr a -> FromIr b
forall a b c. (a -> b -> c) -> FromIr a -> FromIr b -> FromIr c
forall (f :: * -> *).
Functor f
-> (forall a. a -> f a)
-> (forall a b. f (a -> b) -> f a -> f b)
-> (forall a b c. (a -> b -> c) -> f a -> f b -> f c)
-> (forall a b. f a -> f b -> f b)
-> (forall a b. f a -> f b -> f a)
-> Applicative f
$cpure :: forall a. a -> FromIr a
pure :: forall a. a -> FromIr a
$c<*> :: forall a b. FromIr (a -> b) -> FromIr a -> FromIr b
<*> :: forall a b. FromIr (a -> b) -> FromIr a -> FromIr b
$cliftA2 :: forall a b c. (a -> b -> c) -> FromIr a -> FromIr b -> FromIr c
liftA2 :: forall a b c. (a -> b -> c) -> FromIr a -> FromIr b -> FromIr c
$c*> :: forall a b. FromIr a -> FromIr b -> FromIr b
*> :: forall a b. FromIr a -> FromIr b -> FromIr b
$c<* :: forall a b. FromIr a -> FromIr b -> FromIr a
<* :: forall a b. FromIr a -> FromIr b -> FromIr a
Applicative, Applicative FromIr
Applicative FromIr
-> (forall a b. FromIr a -> (a -> FromIr b) -> FromIr b)
-> (forall a b. FromIr a -> FromIr b -> FromIr b)
-> (forall a. a -> FromIr a)
-> Monad FromIr
forall a. a -> FromIr a
forall a b. FromIr a -> FromIr b -> FromIr b
forall a b. FromIr a -> (a -> FromIr b) -> FromIr b
forall (m :: * -> *).
Applicative m
-> (forall a b. m a -> (a -> m b) -> m b)
-> (forall a b. m a -> m b -> m b)
-> (forall a. a -> m a)
-> Monad m
$c>>= :: forall a b. FromIr a -> (a -> FromIr b) -> FromIr b
>>= :: forall a b. FromIr a -> (a -> FromIr b) -> FromIr b
$c>> :: forall a b. FromIr a -> FromIr b -> FromIr b
>> :: forall a b. FromIr a -> FromIr b -> FromIr b
$creturn :: forall a. a -> FromIr a
return :: forall a. a -> FromIr a
Monad, MonadValidate (NonEmpty Error), MonadWriter FromIrWriter)

-- | Collected from using a native query in a query.
--   Each entry here because a CTE to be prepended to the query.
newtype FromIrWriter = FromIrWriter
  { FromIrWriter
-> Map (Aliased NativeQueryName) (InterpolatedQuery Expression)
fromIrWriterNativeQueries :: Map (Aliased NativeQueryName) (InterpolatedQuery Expression)
  }
  deriving newtype (NonEmpty FromIrWriter -> FromIrWriter
FromIrWriter -> FromIrWriter -> FromIrWriter
(FromIrWriter -> FromIrWriter -> FromIrWriter)
-> (NonEmpty FromIrWriter -> FromIrWriter)
-> (forall b. Integral b => b -> FromIrWriter -> FromIrWriter)
-> Semigroup FromIrWriter
forall b. Integral b => b -> FromIrWriter -> FromIrWriter
forall a.
(a -> a -> a)
-> (NonEmpty a -> a)
-> (forall b. Integral b => b -> a -> a)
-> Semigroup a
$c<> :: FromIrWriter -> FromIrWriter -> FromIrWriter
<> :: FromIrWriter -> FromIrWriter -> FromIrWriter
$csconcat :: NonEmpty FromIrWriter -> FromIrWriter
sconcat :: NonEmpty FromIrWriter -> FromIrWriter
$cstimes :: forall b. Integral b => b -> FromIrWriter -> FromIrWriter
stimes :: forall b. Integral b => b -> FromIrWriter -> FromIrWriter
Semigroup, Semigroup FromIrWriter
FromIrWriter
Semigroup FromIrWriter
-> FromIrWriter
-> (FromIrWriter -> FromIrWriter -> FromIrWriter)
-> ([FromIrWriter] -> FromIrWriter)
-> Monoid FromIrWriter
[FromIrWriter] -> FromIrWriter
FromIrWriter -> FromIrWriter -> FromIrWriter
forall a.
Semigroup a -> a -> (a -> a -> a) -> ([a] -> a) -> Monoid a
$cmempty :: FromIrWriter
mempty :: FromIrWriter
$cmappend :: FromIrWriter -> FromIrWriter -> FromIrWriter
mappend :: FromIrWriter -> FromIrWriter -> FromIrWriter
$cmconcat :: [FromIrWriter] -> FromIrWriter
mconcat :: [FromIrWriter] -> FromIrWriter
Monoid)

data FromIrState = FromIrState
  { FromIrState -> Map Text Int
indices :: Map Text Int
  }

data FromIrReader = FromIrReader
  { FromIrReader -> FromIrConfig
config :: FromIrConfig
  }

-- | Config values for the from-IR translator.
data FromIrConfig = FromIrConfig
  { -- | Applies globally to all selects, and may be reduced to
    -- something even smaller by permission/user args.
    FromIrConfig -> Top
globalSelectLimit :: Top
  }

-- | A default config.
defaultFromIrConfig :: FromIrConfig
defaultFromIrConfig :: FromIrConfig
defaultFromIrConfig = FromIrConfig {globalSelectLimit :: Top
globalSelectLimit = Top
NoTop}

-- | Alias of parent SELECT FROM.
-- Functions underlying computed fields requires column values from
-- the table that is being used in FROM clause of parent SELECT.
--
-- Example SQL:
--
-- > SELECT
-- >   `t_author1`.`id` AS `id`,
-- >   `t_author1`.`name` AS `name`,
-- >   ARRAY(
-- >     SELECT
-- >       AS STRUCT `id`,
-- >       `title`,
-- >       `content`
-- >     FROM
-- >       UNNEST(
-- >         ARRAY(
-- >           SELECT
-- >             AS STRUCT *
-- >           FROM `hasura_test`.`fetch_articles`(`id` => `t_author1`.`id`)
-- >         )
-- >       )
-- >       LIMIT 1000
-- >   ) AS `articles`
-- > FROM
-- >   `hasura_test`.`author` AS `t_author1`
--
-- Where `t_author1` is the @'ParentSelectFromIdentity'
data ParentSelectFromEntity
  = -- | There's no parent entity
    NoParentEntity
  | -- | Alias of the parent SELECT FROM
    ParentEntityAlias EntityAlias

--------------------------------------------------------------------------------
-- Runners

runFromIr :: FromIrConfig -> FromIr a -> Validate (NonEmpty Error) (a, FromIrWriter)
runFromIr :: forall a.
FromIrConfig
-> FromIr a -> Validate (NonEmpty Error) (a, FromIrWriter)
runFromIr FromIrConfig
config FromIr a
fromIr =
  WriterT FromIrWriter (Validate (NonEmpty Error)) a
-> ValidateT (NonEmpty Error) Identity (a, FromIrWriter)
forall w (m :: * -> *) a. WriterT w m a -> m (a, w)
runWriterT
    (WriterT FromIrWriter (Validate (NonEmpty Error)) a
 -> ValidateT (NonEmpty Error) Identity (a, FromIrWriter))
-> WriterT FromIrWriter (Validate (NonEmpty Error)) a
-> ValidateT (NonEmpty Error) Identity (a, FromIrWriter)
forall a b. (a -> b) -> a -> b
$ StateT
  FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))) a
-> FromIrState
-> WriterT FromIrWriter (Validate (NonEmpty Error)) a
forall (m :: * -> *) s a. Monad m => StateT s m a -> s -> m a
evalStateT
      (ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  a
-> FromIrReader
-> StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))) a
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT (FromIr a
-> ReaderT
     FromIrReader
     (StateT
        FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
     a
forall a.
FromIr a
-> ReaderT
     FromIrReader
     (StateT
        FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
     a
unFromIr FromIr a
fromIr) (FromIrReader {FromIrConfig
config :: FromIrConfig
config :: FromIrConfig
config}))
      (FromIrState {indices :: Map Text Int
indices = Map Text Int
forall a. Monoid a => a
mempty})

bigQuerySourceConfigToFromIrConfig :: BigQuerySourceConfig -> FromIrConfig
bigQuerySourceConfigToFromIrConfig :: BigQuerySourceConfig -> FromIrConfig
bigQuerySourceConfigToFromIrConfig BigQuerySourceConfig {Int64
_scGlobalSelectLimit :: Int64
_scGlobalSelectLimit :: BigQuerySourceConfig -> Int64
_scGlobalSelectLimit} =
  FromIrConfig {globalSelectLimit :: Top
globalSelectLimit = Int64 -> Top
Top Int64
_scGlobalSelectLimit}

--------------------------------------------------------------------------------
-- Similar rendition of old API

-- | Here is where we apply a top-level annotation to the select to
-- indicate to the data loader that this select ought to produce a
-- single object or an array.
mkSQLSelect ::
  Rql.JsonAggSelect ->
  Ir.AnnSelectG 'BigQuery (Ir.AnnFieldG 'BigQuery Void) Expression ->
  FromIr BigQuery.Select
mkSQLSelect :: JsonAggSelect
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr Select
mkSQLSelect JsonAggSelect
jsonAggSelect AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSimpleSel = do
  Select
select <- PartitionableSelect -> Select
noExtraPartitionFields (PartitionableSelect -> Select)
-> FromIr PartitionableSelect -> FromIr Select
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ParentSelectFromEntity
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr PartitionableSelect
fromSelectRows ParentSelectFromEntity
NoParentEntity AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSimpleSel
  Select -> FromIr Select
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    ( Select
select
        { $sel:selectCardinality:Select :: Cardinality
selectCardinality =
            case JsonAggSelect
jsonAggSelect of
              JsonAggSelect
Rql.JASMultipleRows -> Cardinality
Many
              JsonAggSelect
Rql.JASSingleObject -> Cardinality
One
        }
    )

-- | Convert from the IR database query into a select.
fromRootField :: Ir.QueryDB 'BigQuery Void Expression -> FromIr Select
fromRootField :: QueryDB 'BigQuery Void Expression -> FromIr Select
fromRootField =
  \case
    (Ir.QDBSingleRow AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
s) -> JsonAggSelect
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr Select
mkSQLSelect JsonAggSelect
Rql.JASSingleObject AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
s
    (Ir.QDBMultipleRows AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
s) -> JsonAggSelect
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr Select
mkSQLSelect JsonAggSelect
Rql.JASMultipleRows AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
s
    (Ir.QDBAggregation AnnAggregateSelectG 'BigQuery Void Expression
s) -> Maybe (EntityAlias, HashMap ColumnName ColumnName)
-> AnnAggregateSelectG 'BigQuery Void Expression -> FromIr Select
fromSelectAggregate Maybe (EntityAlias, HashMap ColumnName ColumnName)
forall a. Maybe a
Nothing AnnAggregateSelectG 'BigQuery Void Expression
s

--------------------------------------------------------------------------------
-- Top-level exported functions

fromUnnestedJSON :: Expression -> [(ColumnName, ScalarType)] -> [Rql.FieldName] -> FromIr From
fromUnnestedJSON :: Expression
-> [(ColumnName, ScalarType)] -> [FieldName] -> FromIr From
fromUnnestedJSON Expression
json [(ColumnName, ScalarType)]
columns [FieldName]
_fields = do
  EntityAlias
alias <- NameTemplate -> FromIr EntityAlias
generateEntityAlias NameTemplate
UnnestTemplate
  From -> FromIr From
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    ( Aliased SelectJson -> From
FromSelectJson
        ( Aliased
            { $sel:aliasedThing:Aliased :: SelectJson
aliasedThing =
                SelectJson
                  { $sel:selectJsonBody:SelectJson :: Expression
selectJsonBody = Expression
json,
                    $sel:selectJsonFields:SelectJson :: [(ColumnName, ScalarType)]
selectJsonFields = [(ColumnName, ScalarType)]
columns
                  },
              $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
entityAliasText EntityAlias
alias
            }
        )
    )

fromSelectRows :: ParentSelectFromEntity -> Ir.AnnSelectG 'BigQuery (Ir.AnnFieldG 'BigQuery Void) Expression -> FromIr BigQuery.PartitionableSelect
fromSelectRows :: ParentSelectFromEntity
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr PartitionableSelect
fromSelectRows ParentSelectFromEntity
parentSelectFromEntity AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSelectG = do
  let Ir.AnnSelectG
        { $sel:_asnFields:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> Fields (f v)
_asnFields = Fields (AnnFieldG 'BigQuery Void Expression)
fields,
          $sel:_asnFrom:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> SelectFromG b v
_asnFrom = SelectFromG 'BigQuery Expression
from,
          $sel:_asnPerm:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> TablePermG b v
_asnPerm = TablePermG 'BigQuery Expression
perm,
          $sel:_asnArgs:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> SelectArgsG b v
_asnArgs = SelectArgsG 'BigQuery Expression
args,
          $sel:_asnNamingConvention:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> Maybe NamingCase
_asnNamingConvention = Maybe NamingCase
_tCase
        } = AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSelectG
      Ir.TablePerm {$sel:_tpLimit:TablePerm :: forall (b :: BackendType) v. TablePermG b v -> Maybe Int
_tpLimit = Maybe Int
mPermLimit, $sel:_tpFilter:TablePerm :: forall (b :: BackendType) v. TablePermG b v -> AnnBoolExp b v
_tpFilter = GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
permFilter} = TablePermG 'BigQuery Expression
perm
      permissionBasedTop :: Top
permissionBasedTop =
        Top -> (Int -> Top) -> Maybe Int -> Top
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Top
NoTop (Int64 -> Top
Top (Int64 -> Top) -> (Int -> Int64) -> Int -> Top
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Int -> Int64
forall a b. (Integral a, Num b) => a -> b
fromIntegral) Maybe Int
mPermLimit
  From
selectFrom <-
    case SelectFromG 'BigQuery Expression
from of
      Ir.FromTable TableName 'BigQuery
qualifiedObject -> TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
qualifiedObject
      Ir.FromFunction FunctionName 'BigQuery
nm (Functions.FunctionArgsExp [BigQuery.AEInput Expression
json] HashMap Text (FunctionArgumentExp 'BigQuery Expression)
_) (Just [(Column 'BigQuery, ScalarType 'BigQuery)]
columns)
        | FunctionName -> Text
functionName FunctionName 'BigQuery
FunctionName
nm Text -> Text -> Bool
forall a. Eq a => a -> a -> Bool
== Text
"unnest" -> Expression
-> [(ColumnName, ScalarType)] -> [FieldName] -> FromIr From
fromUnnestedJSON Expression
json [(Column 'BigQuery, ScalarType 'BigQuery)]
[(ColumnName, ScalarType)]
columns (((FieldName, AnnFieldG 'BigQuery Void Expression) -> FieldName)
-> Fields (AnnFieldG 'BigQuery Void Expression) -> [FieldName]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, AnnFieldG 'BigQuery Void Expression) -> FieldName
forall a b. (a, b) -> a
fst Fields (AnnFieldG 'BigQuery Void Expression)
fields)
      Ir.FromFunction FunctionName 'BigQuery
functionName (Functions.FunctionArgsExp [FunctionArgumentExp 'BigQuery Expression]
positionalArgs HashMap Text (FunctionArgumentExp 'BigQuery Expression)
namedArgs) Maybe [(Column 'BigQuery, ScalarType 'BigQuery)]
Nothing ->
        ParentSelectFromEntity
-> FunctionName
-> [ArgumentExp Expression]
-> HashMap Text (ArgumentExp Expression)
-> FromIr From
fromFunction ParentSelectFromEntity
parentSelectFromEntity FunctionName 'BigQuery
FunctionName
functionName [FunctionArgumentExp 'BigQuery Expression]
[ArgumentExp Expression]
positionalArgs HashMap Text (FunctionArgumentExp 'BigQuery Expression)
HashMap Text (ArgumentExp Expression)
namedArgs
      Ir.FromNativeQuery NativeQuery 'BigQuery Expression
nativeQuery -> NativeQuery 'BigQuery Expression -> FromIr From
fromNativeQuery NativeQuery 'BigQuery Expression
nativeQuery
      SelectFromG 'BigQuery Expression
_ -> NonEmpty Error -> FromIr From
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (SelectFromG 'BigQuery Expression -> Error
FromTypeUnsupported SelectFromG 'BigQuery Expression
from))
  Args
    { Maybe (NonEmpty OrderBy)
argsOrderBy :: Maybe (NonEmpty OrderBy)
argsOrderBy :: Args -> Maybe (NonEmpty OrderBy)
argsOrderBy,
      Where
argsWhere :: Where
argsWhere :: Args -> Where
argsWhere,
      [Join]
argsJoins :: [Join]
argsJoins :: Args -> [Join]
argsJoins,
      Top
argsTop :: Top
argsTop :: Args -> Top
argsTop,
      Maybe (NonEmpty ColumnName)
argsDistinct :: Maybe (NonEmpty ColumnName)
argsDistinct :: Args -> Maybe (NonEmpty ColumnName)
argsDistinct,
      Maybe Int64
argsOffset :: Maybe Int64
argsOffset :: Args -> Maybe Int64
argsOffset,
      Map TableName EntityAlias
argsExistingJoins :: Map TableName EntityAlias
argsExistingJoins :: Args -> Map TableName EntityAlias
argsExistingJoins
    } <-
    ReaderT EntityAlias FromIr Args -> EntityAlias -> FromIr Args
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT (SelectArgsG 'BigQuery Expression -> ReaderT EntityAlias FromIr Args
fromSelectArgsG SelectArgsG 'BigQuery Expression
args) (From -> EntityAlias
fromAlias From
selectFrom)
  [FieldSource]
fieldSources <-
    ReaderT EntityAlias FromIr [FieldSource]
-> EntityAlias -> FromIr [FieldSource]
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT
      (((FieldName, AnnFieldG 'BigQuery Void Expression)
 -> ReaderT EntityAlias FromIr FieldSource)
-> Fields (AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr [FieldSource]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse (Map TableName EntityAlias
-> (FieldName, AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr FieldSource
fromAnnFieldsG Map TableName EntityAlias
argsExistingJoins) Fields (AnnFieldG 'BigQuery Void Expression)
fields)
      (From -> EntityAlias
fromAlias From
selectFrom)
  Expression
filterExpression <-
    ReaderT EntityAlias FromIr Expression
-> EntityAlias -> FromIr Expression
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
permFilter) (From -> EntityAlias
fromAlias From
selectFrom)
  NonEmpty Projection
selectProjections <- Bool -> [FieldSource] -> FromIr (NonEmpty Projection)
selectProjectionsFromFieldSources Bool
True [FieldSource]
fieldSources
  Top
globalTop <- FromIr Top
getGlobalTop
  let select :: Select
select =
        Select
          { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
            $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
Many,
            $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
            $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = [Text] -> Maybe [Text]
forall a. a -> Maybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Fields (AnnFieldG 'BigQuery Void Expression) -> [Text]
fieldTextNames Fields (AnnFieldG 'BigQuery Void Expression)
fields),
            $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
            $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
argsOrderBy,
            -- We DO APPLY the global top here, because this pulls down all rows.
            $sel:selectTop:Select :: Top
selectTop = Top
globalTop Top -> Top -> Top
forall a. Semigroup a => a -> a -> a
<> Top
permissionBasedTop Top -> Top -> Top
forall a. Semigroup a => a -> a -> a
<> Top
argsTop,
            NonEmpty Projection
selectProjections :: NonEmpty Projection
$sel:selectProjections:Select :: NonEmpty Projection
selectProjections,
            From
selectFrom :: From
$sel:selectFrom:Select :: From
selectFrom,
            $sel:selectJoins:Select :: [Join]
selectJoins = [Join]
argsJoins [Join] -> [Join] -> [Join]
forall a. Semigroup a => a -> a -> a
<> [[Join]] -> [Join]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
concat ((FieldSource -> Maybe [Join]) -> [FieldSource] -> [[Join]]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe FieldSource -> Maybe [Join]
fieldSourceJoins [FieldSource]
fieldSources),
            $sel:selectWhere:Select :: Where
selectWhere = Where
argsWhere Where -> Where -> Where
forall a. Semigroup a => a -> a -> a
<> [Expression] -> Where
Where [Expression
filterExpression],
            $sel:selectOffset:Select :: Maybe Expression
selectOffset = Int64 -> Expression
int64Expr (Int64 -> Expression) -> Maybe Int64 -> Maybe Expression
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe Int64
argsOffset
          }
  case Maybe (NonEmpty ColumnName)
argsDistinct of
    Maybe (NonEmpty ColumnName)
Nothing ->
      PartitionableSelect -> FromIr PartitionableSelect
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (PartitionableSelect -> FromIr PartitionableSelect)
-> PartitionableSelect -> FromIr PartitionableSelect
forall a b. (a -> b) -> a -> b
$ Select -> PartitionableSelect
simpleSelect Select
select
    Just NonEmpty ColumnName
distinct ->
      Select
-> NonEmpty ColumnName
-> Maybe (NonEmpty OrderBy)
-> FromIr PartitionableSelect
simulateDistinctOn Select
select NonEmpty ColumnName
distinct Maybe (NonEmpty OrderBy)
argsOrderBy

-- | Simulates DISTINCT ON for BigQuery using ROW_NUMBER() partitioned over distinct fields
--
-- Example:
--
-- For a GraphQL query:
-- @
-- hasura_test_article(distinct_on: author_id, order_by: [{author_id: asc}, {created_at: asc}]) {
--   id
--   title
-- }
-- @
--
-- it should produce from a query without a `distinct_on` clause:
--
-- SELECT `id`, `title`
-- FROM `hasura_test`.`article`
-- ORDER BY `author_id` ASC, `created_at` ASC
--
-- a query of the following form:
--
-- SELECT `id`, `title`
-- FROM (SELECT *,
--              ROW_NUMBER() OVER (PARTITION BY `author_id` ORDER BY `created_at` ASC) as `idx1`
--       FROM `hasura_test`.`article`) as `t_article1`
-- WHERE (`t_article1`.`idx1` = 1)
-- ORDER BY `t_article1`.`author_id` ASC
--
-- Note: this method returns PartitionableSelect as it could be joined using an array relation
-- which requires extra fields added to the PARTITION BY clause to return proper results
simulateDistinctOn :: Select -> NonEmpty ColumnName -> Maybe (NonEmpty OrderBy) -> FromIr PartitionableSelect
simulateDistinctOn :: Select
-> NonEmpty ColumnName
-> Maybe (NonEmpty OrderBy)
-> FromIr PartitionableSelect
simulateDistinctOn Select
select NonEmpty ColumnName
distinctOnColumns Maybe (NonEmpty OrderBy)
orderByColumns = do
  EntityAlias
rowNumAlias <- NameTemplate -> FromIr EntityAlias
generateEntityAlias NameTemplate
IndexTemplate
  PartitionableSelect -> FromIr PartitionableSelect
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    PartitionableSelect
      { $sel:pselectFrom:PartitionableSelect :: From
pselectFrom = Select -> From
selectFrom Select
select,
        $sel:pselectFinalize:PartitionableSelect :: Maybe [FieldName] -> Select
pselectFinalize = \Maybe [FieldName]
mExtraPartitionField ->
          let -- we use the same alias both for outer and inner selects
              alias :: Text
alias = EntityAlias -> Text
entityAliasText (From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select))
              distinctFields :: NonEmpty FieldName
distinctFields = (ColumnName -> FieldName)
-> NonEmpty ColumnName -> NonEmpty FieldName
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(ColumnName Text
name) -> Text -> Text -> FieldName
FieldName Text
name Text
alias) NonEmpty ColumnName
distinctOnColumns
              finalDistinctFields :: NonEmpty FieldName
finalDistinctFields = case Maybe [FieldName]
mExtraPartitionField of
                Just [FieldName]
extraFields
                  | Just NonEmpty FieldName
neExtraFields <- [FieldName] -> Maybe (NonEmpty FieldName)
forall a. [a] -> Maybe (NonEmpty a)
nonEmpty [FieldName]
extraFields ->
                      NonEmpty FieldName
neExtraFields NonEmpty FieldName -> NonEmpty FieldName -> NonEmpty FieldName
forall a. Semigroup a => a -> a -> a
<> NonEmpty FieldName
distinctFields
                Maybe [FieldName]
_ -> NonEmpty FieldName
distinctFields
              (Maybe (NonEmpty OrderBy)
distinctOnOrderBy, Maybe (NonEmpty OrderBy)
innerOrderBy) =
                case Maybe (NonEmpty OrderBy)
orderByColumns of
                  Just NonEmpty OrderBy
orderBy ->
                    let ([OrderBy]
distincts, [OrderBy]
others) = (OrderBy -> Bool) -> NonEmpty OrderBy -> ([OrderBy], [OrderBy])
forall a. (a -> Bool) -> NonEmpty a -> ([a], [a])
NE.partition (\OrderBy {FieldName
NullsOrder
Order
orderByFieldName :: FieldName
orderByOrder :: Order
orderByNullsOrder :: NullsOrder
$sel:orderByFieldName:OrderBy :: OrderBy -> FieldName
$sel:orderByOrder:OrderBy :: OrderBy -> Order
$sel:orderByNullsOrder:OrderBy :: OrderBy -> NullsOrder
..} -> FieldName
orderByFieldName FieldName -> NonEmpty FieldName -> Bool
forall a. Eq a => a -> NonEmpty a -> Bool
forall (t :: * -> *) a. (Foldable t, Eq a) => a -> t a -> Bool
`elem` NonEmpty FieldName
distinctFields) NonEmpty OrderBy
orderBy
                     in ([OrderBy] -> Maybe (NonEmpty OrderBy)
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [OrderBy]
distincts, [OrderBy] -> Maybe (NonEmpty OrderBy)
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [OrderBy]
others)
                  Maybe (NonEmpty OrderBy)
Nothing ->
                    (Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing, Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing)
              innerFrom :: From
innerFrom =
                Aliased Select -> From
FromSelect
                  Aliased
                    { $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
alias,
                      $sel:aliasedThing:Aliased :: Select
aliasedThing =
                        Select
select
                          { $sel:selectProjections:Select :: NonEmpty Projection
selectProjections =
                              Projection
StarProjection
                                Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| [ Aliased WindowFunction -> Projection
WindowProjection
                                       ( Aliased
                                           { $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
unEntityAlias EntityAlias
rowNumAlias,
                                             $sel:aliasedThing:Aliased :: WindowFunction
aliasedThing =
                                               NonEmpty FieldName -> Maybe (NonEmpty OrderBy) -> WindowFunction
RowNumberOverPartitionBy
                                                 NonEmpty FieldName
finalDistinctFields
                                                 Maybe (NonEmpty OrderBy)
innerOrderBy
                                                 -- Above: Having the order by
                                                 -- in here ensures that we get the proper
                                                 -- row as the first one we select
                                                 -- in the outer select WHERE condition
                                                 -- to simulate DISTINCT ON semantics
                                           }
                                       )
                                   ],
                            $sel:selectTop:Select :: Top
selectTop = Top
forall a. Monoid a => a
mempty,
                            $sel:selectJoins:Select :: [Join]
selectJoins = [Join]
forall a. Monoid a => a
mempty,
                            $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Monoid a => a
mempty,
                            $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing,
                            $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
                            $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Monoid a => a
mempty
                          }
                    }
           in Select
select
                { $sel:selectFrom:Select :: From
selectFrom = From
innerFrom,
                  $sel:selectWhere:Select :: Where
selectWhere =
                    [Expression] -> Where
Where
                      [ Expression -> Expression -> Expression
EqualExpression
                          (FieldName -> Expression
ColumnExpression FieldName {$sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = Text
alias, $sel:fieldName:FieldName :: Text
fieldName = EntityAlias -> Text
unEntityAlias EntityAlias
rowNumAlias})
                          (Int64 -> Expression
int64Expr Int64
1)
                      ],
                  $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
distinctOnOrderBy
                }
      }

fromSelectAggregate ::
  Maybe (EntityAlias, HashMap ColumnName ColumnName) ->
  Ir.AnnSelectG 'BigQuery (Ir.TableAggregateFieldG 'BigQuery Void) Expression ->
  FromIr BigQuery.Select
fromSelectAggregate :: Maybe (EntityAlias, HashMap ColumnName ColumnName)
-> AnnAggregateSelectG 'BigQuery Void Expression -> FromIr Select
fromSelectAggregate Maybe (EntityAlias, HashMap ColumnName ColumnName)
minnerJoinFields AnnAggregateSelectG 'BigQuery Void Expression
annSelectG = do
  From
selectFrom <-
    case SelectFromG 'BigQuery Expression
from of
      Ir.FromTable TableName 'BigQuery
qualifiedObject -> TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
qualifiedObject
      SelectFromG 'BigQuery Expression
_ -> NonEmpty Error -> FromIr From
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (SelectFromG 'BigQuery Expression -> Error
FromTypeUnsupported SelectFromG 'BigQuery Expression
from))
  args' :: Args
args'@Args {Where
argsWhere :: Args -> Where
argsWhere :: Where
argsWhere, Maybe (NonEmpty OrderBy)
argsOrderBy :: Args -> Maybe (NonEmpty OrderBy)
argsOrderBy :: Maybe (NonEmpty OrderBy)
argsOrderBy, [Join]
argsJoins :: Args -> [Join]
argsJoins :: [Join]
argsJoins, Top
argsTop :: Args -> Top
argsTop :: Top
argsTop, Maybe Int64
argsOffset :: Args -> Maybe Int64
argsOffset :: Maybe Int64
argsOffset, Maybe (NonEmpty ColumnName)
argsDistinct :: Args -> Maybe (NonEmpty ColumnName)
argsDistinct :: Maybe (NonEmpty ColumnName)
argsDistinct} <-
    ReaderT EntityAlias FromIr Args -> EntityAlias -> FromIr Args
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT (SelectArgsG 'BigQuery Expression -> ReaderT EntityAlias FromIr Args
fromSelectArgsG SelectArgsG 'BigQuery Expression
args) (From -> EntityAlias
fromAlias From
selectFrom)
  Expression
filterExpression <-
    ReaderT EntityAlias FromIr Expression
-> EntityAlias -> FromIr Expression
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
permFilter) (From -> EntityAlias
fromAlias From
selectFrom)
  Maybe [(FieldName, FieldName)]
mforeignKeyConditions <-
    Maybe (EntityAlias, HashMap ColumnName ColumnName)
-> ((EntityAlias, HashMap ColumnName ColumnName)
    -> FromIr [(FieldName, FieldName)])
-> FromIr (Maybe [(FieldName, FieldName)])
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
t a -> (a -> f b) -> f (t b)
for Maybe (EntityAlias, HashMap ColumnName ColumnName)
minnerJoinFields (((EntityAlias, HashMap ColumnName ColumnName)
  -> FromIr [(FieldName, FieldName)])
 -> FromIr (Maybe [(FieldName, FieldName)]))
-> ((EntityAlias, HashMap ColumnName ColumnName)
    -> FromIr [(FieldName, FieldName)])
-> FromIr (Maybe [(FieldName, FieldName)])
forall a b. (a -> b) -> a -> b
$ \(EntityAlias
entityAlias, HashMap ColumnName ColumnName
mapping) ->
      ReaderT EntityAlias FromIr [(FieldName, FieldName)]
-> EntityAlias -> FromIr [(FieldName, FieldName)]
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT
        (EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames (From -> EntityAlias
fromAlias From
selectFrom) HashMap ColumnName ColumnName
mapping)
        EntityAlias
entityAlias
  [FieldSource]
fieldSources <-
    ReaderT EntityAlias FromIr [FieldSource]
-> EntityAlias -> FromIr [FieldSource]
forall r (m :: * -> *) a. ReaderT r m a -> r -> m a
runReaderT
      ( ((FieldName, TableAggregateFieldG 'BigQuery Void Expression)
 -> ReaderT EntityAlias FromIr FieldSource)
-> [(FieldName, TableAggregateFieldG 'BigQuery Void Expression)]
-> ReaderT EntityAlias FromIr [FieldSource]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse
          ( Args
-> Top
-> (FieldName, TableAggregateFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr FieldSource
fromTableAggregateFieldG
              Args
args'
              Top
permissionBasedTop
          )
          [(FieldName, TableAggregateFieldG 'BigQuery Void Expression)]
fields
      )
      (From -> EntityAlias
fromAlias From
selectFrom)
  NonEmpty Projection
selectProjections <- Bool -> [FieldSource] -> FromIr (NonEmpty Projection)
selectProjectionsFromFieldSources Bool
True [FieldSource]
fieldSources
  EntityAlias
indexAlias <- NameTemplate -> FromIr EntityAlias
generateEntityAlias NameTemplate
IndexTemplate
  let innerSelectAlias :: Text
innerSelectAlias = EntityAlias -> Text
entityAliasText (From -> EntityAlias
fromAlias From
selectFrom)
      mDistinctFields :: Maybe (NonEmpty FieldName)
mDistinctFields = (NonEmpty ColumnName -> NonEmpty FieldName)
-> Maybe (NonEmpty ColumnName) -> Maybe (NonEmpty FieldName)
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ((ColumnName -> FieldName)
-> NonEmpty ColumnName -> NonEmpty FieldName
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(ColumnName Text
name) -> Text -> Text -> FieldName
FieldName Text
name Text
innerSelectAlias)) Maybe (NonEmpty ColumnName)
argsDistinct
      mPartitionFields :: Maybe (NonEmpty FieldName)
mPartitionFields = (Maybe [(FieldName, FieldName)]
mforeignKeyConditions Maybe [(FieldName, FieldName)]
-> ([(FieldName, FieldName)] -> Maybe (NonEmpty FieldName))
-> Maybe (NonEmpty FieldName)
forall a b. Maybe a -> (a -> Maybe b) -> Maybe b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= [FieldName] -> Maybe (NonEmpty FieldName)
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty ([FieldName] -> Maybe (NonEmpty FieldName))
-> ([(FieldName, FieldName)] -> [FieldName])
-> [(FieldName, FieldName)]
-> Maybe (NonEmpty FieldName)
forall b c a. (b -> c) -> (a -> b) -> a -> c
. ((FieldName, FieldName) -> FieldName)
-> [(FieldName, FieldName)] -> [FieldName]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, FieldName) -> FieldName
forall a b. (a, b) -> a
fst) Maybe (NonEmpty FieldName)
-> Maybe (NonEmpty FieldName) -> Maybe (NonEmpty FieldName)
forall a. Semigroup a => a -> a -> a
<> Maybe (NonEmpty FieldName)
mDistinctFields
      innerProjections :: NonEmpty Projection
innerProjections =
        case Maybe (NonEmpty FieldName)
mPartitionFields of
          Maybe (NonEmpty FieldName)
Nothing -> Projection -> NonEmpty Projection
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Projection
StarProjection
          Just NonEmpty FieldName
partitionFields ->
            Projection
StarProjection
              Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:|
              -- We setup an index over every row in
              -- the sub select.  Then if you look at
              -- the outer Select, you can see we apply
              -- a WHERE that uses this index for
              -- LIMIT/OFFSET or DISTINCT ON.
              [ Aliased WindowFunction -> Projection
WindowProjection
                  ( Aliased
                      { $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
unEntityAlias EntityAlias
indexAlias,
                        $sel:aliasedThing:Aliased :: WindowFunction
aliasedThing =
                          NonEmpty FieldName -> Maybe (NonEmpty OrderBy) -> WindowFunction
RowNumberOverPartitionBy
                            -- The row numbers start from 1.
                            NonEmpty FieldName
partitionFields
                            Maybe (NonEmpty OrderBy)
argsOrderBy
                            -- Above: Having the order by
                            -- in here ensures that the
                            -- row numbers are ordered by
                            -- this ordering. Below, we
                            -- order again for the
                            -- general row order. Both
                            -- are needed!
                      }
                  )
              ]
      indexColumn :: Expression
indexColumn =
        FieldName -> Expression
ColumnExpression
          (FieldName -> Expression) -> FieldName -> Expression
forall a b. (a -> b) -> a -> b
$ FieldName
            { $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = Text
innerSelectAlias,
              $sel:fieldName:FieldName :: Text
fieldName = EntityAlias -> Text
unEntityAlias EntityAlias
indexAlias
            }
  Select -> FromIr Select
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    Select
      { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
        $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
        $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
        $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
        $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
        NonEmpty Projection
$sel:selectProjections:Select :: NonEmpty Projection
selectProjections :: NonEmpty Projection
selectProjections,
        $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
        $sel:selectFrom:Select :: From
selectFrom =
          Aliased Select -> From
FromSelect
            ( Aliased
                { $sel:aliasedThing:Aliased :: Select
aliasedThing =
                    Select
                      { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                        $sel:selectProjections:Select :: NonEmpty Projection
selectProjections = NonEmpty Projection
innerProjections,
                        $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
                        From
$sel:selectFrom:Select :: From
selectFrom :: From
selectFrom,
                        $sel:selectJoins:Select :: [Join]
selectJoins = [Join]
argsJoins,
                        $sel:selectWhere:Select :: Where
selectWhere = Where
argsWhere Where -> Where -> Where
forall a. Semigroup a => a -> a -> a
<> ([Expression] -> Where
Where [Expression
filterExpression]),
                        $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
argsOrderBy,
                        -- Above: This is important to have here, because
                        -- offset/top apply AFTER ordering is applied, so
                        -- you can't put an order by in afterwards in a
                        -- parent query. Therefore be careful about
                        -- putting this elsewhere.
                        $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
                        $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
Many,
                        $sel:selectTop:Select :: Top
selectTop = Top
-> ([(FieldName, FieldName)] -> Top)
-> Maybe [(FieldName, FieldName)]
-> Top
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Top
argsTop (Top -> [(FieldName, FieldName)] -> Top
forall a b. a -> b -> a
const Top
NoTop) Maybe [(FieldName, FieldName)]
mforeignKeyConditions,
                        -- we apply offset only if we don't have partitions
                        -- when we do OFFSET/LIMIT based on ROW_NUMBER()
                        $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
-> (NonEmpty FieldName -> Maybe Expression)
-> Maybe (NonEmpty FieldName)
-> Maybe Expression
forall b a. b -> (a -> b) -> Maybe a -> b
maybe (Int64 -> Expression
int64Expr (Int64 -> Expression) -> Maybe Int64 -> Maybe Expression
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe Int64
argsOffset) (Maybe Expression -> NonEmpty FieldName -> Maybe Expression
forall a b. a -> b -> a
const Maybe Expression
forall a. Maybe a
Nothing) Maybe (NonEmpty FieldName)
mPartitionFields,
                        $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty
                      },
                  $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
innerSelectAlias
                }
            ),
        $sel:selectJoins:Select :: [Join]
selectJoins = [[Join]] -> [Join]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
concat ((FieldSource -> Maybe [Join]) -> [FieldSource] -> [[Join]]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe FieldSource -> Maybe [Join]
fieldSourceJoins [FieldSource]
fieldSources),
        $sel:selectWhere:Select :: Where
selectWhere =
          case Maybe (NonEmpty FieldName)
mPartitionFields of
            Maybe (NonEmpty FieldName)
Nothing -> Where
forall a. Monoid a => a
mempty
            Just {} ->
              let offset :: [Expression]
offset =
                    case Maybe (NonEmpty ColumnName)
argsDistinct of
                      Maybe (NonEmpty ColumnName)
Nothing ->
                        case Maybe Int64
argsOffset of
                          Maybe Int64
Nothing -> [Expression]
forall a. Monoid a => a
mempty
                          Just Int64
offset' ->
                            -- Apply an offset using the row_number from above.
                            [ Op -> Expression -> Expression -> Expression
OpExpression
                                Op
MoreOp
                                Expression
indexColumn
                                (Int64 -> Expression
int64Expr Int64
offset')
                            ]
                      Just {} ->
                        -- in case of distinct_on we need to select the row number offset+1
                        -- effectively skipping number of rows equal to offset
                        [ Expression -> Expression -> Expression
EqualExpression
                            Expression
indexColumn
                            (Int64 -> Expression
int64Expr (Int64 -> Maybe Int64 -> Int64
forall a. a -> Maybe a -> a
fromMaybe Int64
0 Maybe Int64
argsOffset Int64 -> Int64 -> Int64
forall a. Num a => a -> a -> a
+ Int64
1))
                        ]
                  limit :: [Expression]
limit =
                    case Top
argsTop of
                      Top
NoTop -> [Expression]
forall a. Monoid a => a
mempty
                      Top Int64
limit' ->
                        -- Apply a limit using the row_number from above.
                        [ Op -> Expression -> Expression -> Expression
OpExpression
                            Op
LessOp
                            Expression
indexColumn
                            ( Int64 -> Expression
int64Expr (Int64
limit' Int64 -> Int64 -> Int64
forall a. Num a => a -> a -> a
+ Int64
1) -- Because the row_number() indexing starts at 1.
                            -- So idx<l+1  means idx<2 where l = 1 i.e. "limit to 1 row".
                            )
                        ]
               in [Expression] -> Where
Where ([Expression]
offset [Expression] -> [Expression] -> [Expression]
forall a. Semigroup a => a -> a -> a
<> [Expression]
limit),
        $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
        $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing
      }
  where
    Ir.AnnSelectG
      { $sel:_asnFields:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> Fields (f v)
_asnFields = [(FieldName, TableAggregateFieldG 'BigQuery Void Expression)]
fields,
        $sel:_asnFrom:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> SelectFromG b v
_asnFrom = SelectFromG 'BigQuery Expression
from,
        $sel:_asnPerm:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> TablePermG b v
_asnPerm = TablePermG 'BigQuery Expression
perm,
        $sel:_asnArgs:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> SelectArgsG b v
_asnArgs = SelectArgsG 'BigQuery Expression
args,
        $sel:_asnNamingConvention:AnnSelectG :: forall (b :: BackendType) (f :: * -> *) v.
AnnSelectG b f v -> Maybe NamingCase
_asnNamingConvention = Maybe NamingCase
_tCase
      } = AnnAggregateSelectG 'BigQuery Void Expression
annSelectG
    Ir.TablePerm {$sel:_tpLimit:TablePerm :: forall (b :: BackendType) v. TablePermG b v -> Maybe Int
_tpLimit = Maybe Int
mPermLimit, $sel:_tpFilter:TablePerm :: forall (b :: BackendType) v. TablePermG b v -> AnnBoolExp b v
_tpFilter = GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
permFilter} = TablePermG 'BigQuery Expression
perm
    permissionBasedTop :: Top
permissionBasedTop =
      Top -> (Int -> Top) -> Maybe Int -> Top
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Top
NoTop (Int64 -> Top
Top (Int64 -> Top) -> (Int -> Int64) -> Int -> Top
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Int -> Int64
forall a b. (Integral a, Num b) => a -> b
fromIntegral) Maybe Int
mPermLimit

--------------------------------------------------------------------------------
-- GraphQL Args

data Args = Args
  { Args -> Where
argsWhere :: Where,
    Args -> Maybe (NonEmpty OrderBy)
argsOrderBy :: Maybe (NonEmpty OrderBy),
    Args -> [Join]
argsJoins :: [Join],
    Args -> Top
argsTop :: Top,
    Args -> Maybe Int64
argsOffset :: Maybe Int.Int64,
    Args -> Maybe (NonEmpty ColumnName)
argsDistinct :: Maybe (NonEmpty ColumnName),
    Args -> Map TableName EntityAlias
argsExistingJoins :: Map TableName EntityAlias
  }
  deriving (Int -> Args -> ShowS
[Args] -> ShowS
Args -> [Char]
(Int -> Args -> ShowS)
-> (Args -> [Char]) -> ([Args] -> ShowS) -> Show Args
forall a.
(Int -> a -> ShowS) -> (a -> [Char]) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> Args -> ShowS
showsPrec :: Int -> Args -> ShowS
$cshow :: Args -> [Char]
show :: Args -> [Char]
$cshowList :: [Args] -> ShowS
showList :: [Args] -> ShowS
Show)

data UnfurledJoin = UnfurledJoin
  { UnfurledJoin -> Join
unfurledJoin :: Join,
    -- | Recorded if we joined onto an object relation.
    UnfurledJoin -> Maybe (TableName, EntityAlias)
unfurledObjectTableAlias :: Maybe (TableName, EntityAlias)
  }
  deriving (Int -> UnfurledJoin -> ShowS
[UnfurledJoin] -> ShowS
UnfurledJoin -> [Char]
(Int -> UnfurledJoin -> ShowS)
-> (UnfurledJoin -> [Char])
-> ([UnfurledJoin] -> ShowS)
-> Show UnfurledJoin
forall a.
(Int -> a -> ShowS) -> (a -> [Char]) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> UnfurledJoin -> ShowS
showsPrec :: Int -> UnfurledJoin -> ShowS
$cshow :: UnfurledJoin -> [Char]
show :: UnfurledJoin -> [Char]
$cshowList :: [UnfurledJoin] -> ShowS
showList :: [UnfurledJoin] -> ShowS
Show)

fromSelectArgsG :: Ir.SelectArgsG 'BigQuery Expression -> ReaderT EntityAlias FromIr Args
fromSelectArgsG :: SelectArgsG 'BigQuery Expression -> ReaderT EntityAlias FromIr Args
fromSelectArgsG SelectArgsG 'BigQuery Expression
selectArgsG = do
  Where
argsWhere <-
    ReaderT EntityAlias FromIr Where
-> (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
    -> ReaderT EntityAlias FromIr Where)
-> Maybe (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression))
-> ReaderT EntityAlias FromIr Where
forall b a. b -> (a -> b) -> Maybe a -> b
maybe (Where -> ReaderT EntityAlias FromIr Where
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Where
forall a. Monoid a => a
mempty) ((Expression -> Where)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Where
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ([Expression] -> Where
Where ([Expression] -> Where)
-> (Expression -> [Expression]) -> Expression -> Where
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Expression -> [Expression]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure) (ReaderT EntityAlias FromIr Expression
 -> ReaderT EntityAlias FromIr Where)
-> (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
    -> ReaderT EntityAlias FromIr Expression)
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Where
forall b c a. (b -> c) -> (a -> b) -> a -> c
. GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp) Maybe (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression))
mannBoolExp
  let argsTop :: Top
argsTop = Top -> (Int -> Top) -> Maybe Int -> Top
forall b a. b -> (a -> b) -> Maybe a -> b
maybe Top
forall a. Monoid a => a
mempty (Int64 -> Top
Top (Int64 -> Top) -> (Int -> Int64) -> Int -> Top
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Int -> Int64
forall a b. (Integral a, Num b) => a -> b
fromIntegral) Maybe Int
mlimit
  ([OrderBy]
argsOrderBy, Seq UnfurledJoin
joins) <-
    WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) [OrderBy]
-> ReaderT EntityAlias FromIr ([OrderBy], Seq UnfurledJoin)
forall w (m :: * -> *) a. WriterT w m a -> m (a, w)
runWriterT ((AnnotatedOrderByItemG 'BigQuery Expression
 -> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) OrderBy)
-> [AnnotatedOrderByItemG 'BigQuery Expression]
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) [OrderBy]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse AnnotatedOrderByItemG 'BigQuery Expression
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) OrderBy
fromAnnotatedOrderByItemG ([AnnotatedOrderByItemG 'BigQuery Expression]
-> (NonEmpty (AnnotatedOrderByItemG 'BigQuery Expression)
    -> [AnnotatedOrderByItemG 'BigQuery Expression])
-> Maybe (NonEmpty (AnnotatedOrderByItemG 'BigQuery Expression))
-> [AnnotatedOrderByItemG 'BigQuery Expression]
forall b a. b -> (a -> b) -> Maybe a -> b
maybe [] NonEmpty (AnnotatedOrderByItemG 'BigQuery Expression)
-> [AnnotatedOrderByItemG 'BigQuery Expression]
forall a. NonEmpty a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList Maybe (NonEmpty (AnnotatedOrderByItemG 'BigQuery Expression))
orders))
  -- Any object-relation joins that we generated, we record their
  -- generated names into a mapping.
  let argsExistingJoins :: Map TableName EntityAlias
argsExistingJoins =
        [(TableName, EntityAlias)] -> Map TableName EntityAlias
forall k a. Ord k => [(k, a)] -> Map k a
M.fromList ((UnfurledJoin -> Maybe (TableName, EntityAlias))
-> [UnfurledJoin] -> [(TableName, EntityAlias)]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe UnfurledJoin -> Maybe (TableName, EntityAlias)
unfurledObjectTableAlias (Seq UnfurledJoin -> [UnfurledJoin]
forall a. Seq a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList Seq UnfurledJoin
joins))
  Args -> ReaderT EntityAlias FromIr Args
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    Args
      { argsJoins :: [Join]
argsJoins = Seq Join -> [Join]
forall a. Seq a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList ((UnfurledJoin -> Join) -> Seq UnfurledJoin -> Seq Join
forall a b. (a -> b) -> Seq a -> Seq b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap UnfurledJoin -> Join
unfurledJoin Seq UnfurledJoin
joins),
        argsOrderBy :: Maybe (NonEmpty OrderBy)
argsOrderBy = [OrderBy] -> Maybe (NonEmpty OrderBy)
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [OrderBy]
argsOrderBy,
        -- TODO(redactionExp): Deal with the redaction expressions in distinct
        argsDistinct :: Maybe (NonEmpty ColumnName)
argsDistinct = (AnnDistinctColumn 'BigQuery Expression -> ColumnName)
-> NonEmpty (AnnDistinctColumn 'BigQuery Expression)
-> NonEmpty ColumnName
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap AnnDistinctColumn 'BigQuery Expression -> Column 'BigQuery
AnnDistinctColumn 'BigQuery Expression -> ColumnName
forall (b :: BackendType) v. AnnDistinctColumn b v -> Column b
Ir._adcColumn (NonEmpty (AnnDistinctColumn 'BigQuery Expression)
 -> NonEmpty ColumnName)
-> Maybe (NonEmpty (AnnDistinctColumn 'BigQuery Expression))
-> Maybe (NonEmpty ColumnName)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Maybe (NonEmpty (AnnDistinctColumn 'BigQuery Expression))
mdistinct,
        Maybe Int64
Map TableName EntityAlias
Top
Where
argsWhere :: Where
argsTop :: Top
argsOffset :: Maybe Int64
argsExistingJoins :: Map TableName EntityAlias
argsWhere :: Where
argsTop :: Top
argsExistingJoins :: Map TableName EntityAlias
argsOffset :: Maybe Int64
..
      }
  where
    Ir.SelectArgs
      { $sel:_saWhere:SelectArgs :: forall (b :: BackendType) v.
SelectArgsG b v -> Maybe (AnnBoolExp b v)
_saWhere = Maybe (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression))
mannBoolExp,
        $sel:_saLimit:SelectArgs :: forall (b :: BackendType) v. SelectArgsG b v -> Maybe Int
_saLimit = Maybe Int
mlimit,
        $sel:_saOffset:SelectArgs :: forall (b :: BackendType) v. SelectArgsG b v -> Maybe Int64
_saOffset = Maybe Int64
argsOffset,
        $sel:_saDistinct:SelectArgs :: forall (b :: BackendType) v.
SelectArgsG b v -> Maybe (NonEmpty (AnnDistinctColumn b v))
_saDistinct = Maybe (NonEmpty (AnnDistinctColumn 'BigQuery Expression))
mdistinct,
        $sel:_saOrderBy:SelectArgs :: forall (b :: BackendType) v.
SelectArgsG b v -> Maybe (NonEmpty (AnnotatedOrderByItemG b v))
_saOrderBy = Maybe (NonEmpty (AnnotatedOrderByItemG 'BigQuery Expression))
orders
      } = SelectArgsG 'BigQuery Expression
selectArgsG

-- | Produce a valid ORDER BY construct, telling about any joins
-- needed on the side.
fromAnnotatedOrderByItemG ::
  Ir.AnnotatedOrderByItemG 'BigQuery Expression -> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) OrderBy
fromAnnotatedOrderByItemG :: AnnotatedOrderByItemG 'BigQuery Expression
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) OrderBy
fromAnnotatedOrderByItemG Ir.OrderByItemG {Maybe (BasicOrderType 'BigQuery)
obiType :: Maybe (BasicOrderType 'BigQuery)
obiType :: forall (b :: BackendType) a.
OrderByItemG b a -> Maybe (BasicOrderType b)
obiType, AnnotatedOrderByElement 'BigQuery Expression
obiColumn :: AnnotatedOrderByElement 'BigQuery Expression
obiColumn :: forall (b :: BackendType) a. OrderByItemG b a -> a
obiColumn, Maybe (NullsOrderType 'BigQuery)
obiNulls :: Maybe (NullsOrderType 'BigQuery)
obiNulls :: forall (b :: BackendType) a.
OrderByItemG b a -> Maybe (NullsOrderType b)
obiNulls} = do
  FieldName
orderByFieldName <- AnnotatedOrderByElement 'BigQuery Expression
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
unfurlAnnotatedOrderByElement AnnotatedOrderByElement 'BigQuery Expression
obiColumn
  let morderByOrder :: Maybe (BasicOrderType 'BigQuery)
morderByOrder =
        Maybe (BasicOrderType 'BigQuery)
obiType
  let orderByNullsOrder :: NullsOrder
orderByNullsOrder =
        NullsOrder -> Maybe NullsOrder -> NullsOrder
forall a. a -> Maybe a -> a
fromMaybe NullsOrder
NullsAnyOrder Maybe (NullsOrderType 'BigQuery)
Maybe NullsOrder
obiNulls
  case Maybe (BasicOrderType 'BigQuery)
morderByOrder of
    Just BasicOrderType 'BigQuery
orderByOrder -> OrderBy
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) OrderBy
forall a.
a -> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) a
forall (f :: * -> *) a. Applicative f => a -> f a
pure OrderBy {BasicOrderType 'BigQuery
FieldName
NullsOrder
Order
$sel:orderByFieldName:OrderBy :: FieldName
$sel:orderByOrder:OrderBy :: Order
$sel:orderByNullsOrder:OrderBy :: NullsOrder
orderByFieldName :: FieldName
orderByNullsOrder :: NullsOrder
orderByOrder :: BasicOrderType 'BigQuery
..}
    Maybe (BasicOrderType 'BigQuery)
Nothing -> NonEmpty Error
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) OrderBy
forall a.
NonEmpty Error
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
NoOrderSpecifiedInOrderBy)

-- | Unfurl the nested set of object relations (tell'd in the writer)
-- that are terminated by field name (Ir.AOCColumn and
-- Ir.AOCArrayAggregation).
unfurlAnnotatedOrderByElement ::
  Ir.AnnotatedOrderByElement 'BigQuery Expression -> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
unfurlAnnotatedOrderByElement :: AnnotatedOrderByElement 'BigQuery Expression
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
unfurlAnnotatedOrderByElement =
  \case
    Ir.AOCColumn ColumnInfo 'BigQuery
columnInfo AnnRedactionExp 'BigQuery Expression
_redactionExp -> ReaderT EntityAlias FromIr FieldName
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (ColumnInfo 'BigQuery -> ReaderT EntityAlias FromIr FieldName
fromColumnInfo ColumnInfo 'BigQuery
columnInfo) -- TODO(redactionExp): Use this redaction expression
    Ir.AOCObjectRelation Rql.RelInfo {riTarget :: forall (b :: BackendType). RelInfo b -> RelTarget b
riTarget = Rql.RelTargetNativeQuery NativeQueryName
_} GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
_annBoolExp AnnotatedOrderByElement 'BigQuery Expression
_annOrderByElementG ->
      [Char]
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall a. HasCallStack => [Char] -> a
error [Char]
"unfurlAnnotatedOrderByElement RelTargetNativeQuery"
    Ir.AOCObjectRelation Rql.RelInfo {riMapping :: forall (b :: BackendType).
RelInfo b -> HashMap (Column b) (Column b)
riMapping = HashMap (Column 'BigQuery) (Column 'BigQuery)
mapping, riTarget :: forall (b :: BackendType). RelInfo b -> RelTarget b
riTarget = Rql.RelTargetTable TableName 'BigQuery
tableName} GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
annBoolExp AnnotatedOrderByElement 'BigQuery Expression
annOrderByElementG -> do
      From
selectFrom <- ReaderT EntityAlias FromIr From
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) From
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr From -> ReaderT EntityAlias FromIr From
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
tableName))
      EntityAlias
joinAliasEntity <-
        ReaderT EntityAlias FromIr EntityAlias
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) EntityAlias
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr EntityAlias -> ReaderT EntityAlias FromIr EntityAlias
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (NameTemplate -> FromIr EntityAlias
generateEntityAlias (Text -> NameTemplate
ForOrderAlias (TableName -> Text
tableNameText TableName 'BigQuery
TableName
tableName))))
      [(FieldName, FieldName)]
joinOn <- ReaderT EntityAlias FromIr [(FieldName, FieldName)]
-> WriterT
     (Seq UnfurledJoin)
     (ReaderT EntityAlias FromIr)
     [(FieldName, FieldName)]
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames EntityAlias
joinAliasEntity HashMap (Column 'BigQuery) (Column 'BigQuery)
HashMap ColumnName ColumnName
mapping)
      Expression
whereExpression <-
        ReaderT EntityAlias FromIr Expression
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) Expression
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift ((EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Expression
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const (From -> EntityAlias
fromAlias From
selectFrom)) (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
annBoolExp))
      Seq UnfurledJoin
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) ()
forall w (m :: * -> *). MonadWriter w m => w -> m ()
tell
        ( UnfurledJoin -> Seq UnfurledJoin
forall a. a -> Seq a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
            UnfurledJoin
              { unfurledJoin :: Join
unfurledJoin =
                  Join
                    { $sel:joinSource:Join :: JoinSource
joinSource =
                        Select -> JoinSource
JoinSelect
                          Select
                            { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                              $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
                              $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
                              $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
                              $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
                              $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
                              $sel:selectProjections:Select :: NonEmpty Projection
selectProjections = Projection
StarProjection Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| [],
                              From
$sel:selectFrom:Select :: From
selectFrom :: From
selectFrom,
                              $sel:selectJoins:Select :: [Join]
selectJoins = [],
                              $sel:selectWhere:Select :: Where
selectWhere = [Expression] -> Where
Where ([Expression
whereExpression]),
                              $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
                              $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing
                            },
                      $sel:joinRightTable:Join :: EntityAlias
joinRightTable = From -> EntityAlias
fromAlias From
selectFrom,
                      $sel:joinAlias:Join :: EntityAlias
joinAlias = EntityAlias
joinAliasEntity,
                      [(FieldName, FieldName)]
joinOn :: [(FieldName, FieldName)]
$sel:joinOn:Join :: [(FieldName, FieldName)]
joinOn,
                      $sel:joinProvenance:Join :: JoinProvenance
joinProvenance = JoinProvenance
OrderByJoinProvenance,
                      $sel:joinFieldName:Join :: Text
joinFieldName = TableName -> Text
tableNameText TableName 'BigQuery
TableName
tableName, -- TODO: not needed.
                      $sel:joinExtractPath:Join :: Maybe Text
joinExtractPath = Maybe Text
forall a. Maybe a
Nothing,
                      $sel:joinType:Join :: JoinType
joinType = JoinType
LeftOuter
                    },
                unfurledObjectTableAlias :: Maybe (TableName, EntityAlias)
unfurledObjectTableAlias = (TableName, EntityAlias) -> Maybe (TableName, EntityAlias)
forall a. a -> Maybe a
Just (TableName 'BigQuery
TableName
tableName, EntityAlias
joinAliasEntity)
              }
        )
      (EntityAlias -> EntityAlias)
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall a.
(EntityAlias -> EntityAlias)
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) a
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const EntityAlias
joinAliasEntity) (AnnotatedOrderByElement 'BigQuery Expression
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
unfurlAnnotatedOrderByElement AnnotatedOrderByElement 'BigQuery Expression
annOrderByElementG)
    Ir.AOCArrayAggregation Rql.RelInfo {riTarget :: forall (b :: BackendType). RelInfo b -> RelTarget b
riTarget = Rql.RelTargetNativeQuery NativeQueryName
_} GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
_annBoolExp AnnotatedAggregateOrderBy 'BigQuery Expression
_annAggregateOrderBy ->
      [Char]
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall a. HasCallStack => [Char] -> a
error [Char]
"unfurlAnnotatedOrderByElement RelTargetNativeQuery"
    Ir.AOCArrayAggregation Rql.RelInfo {riMapping :: forall (b :: BackendType).
RelInfo b -> HashMap (Column b) (Column b)
riMapping = HashMap (Column 'BigQuery) (Column 'BigQuery)
mapping, riTarget :: forall (b :: BackendType). RelInfo b -> RelTarget b
riTarget = Rql.RelTargetTable TableName 'BigQuery
tableName} GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
annBoolExp AnnotatedAggregateOrderBy 'BigQuery Expression
annAggregateOrderBy -> do
      From
selectFrom <- ReaderT EntityAlias FromIr From
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) From
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr From -> ReaderT EntityAlias FromIr From
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
tableName))
      let alias :: Text
alias = Text
aggFieldName
      EntityAlias
joinAlias <-
        ReaderT EntityAlias FromIr EntityAlias
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) EntityAlias
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr EntityAlias -> ReaderT EntityAlias FromIr EntityAlias
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (NameTemplate -> FromIr EntityAlias
generateEntityAlias (Text -> NameTemplate
ForOrderAlias (TableName -> Text
tableNameText TableName 'BigQuery
TableName
tableName))))
      [(FieldName, FieldName)]
joinOn <- ReaderT EntityAlias FromIr [(FieldName, FieldName)]
-> WriterT
     (Seq UnfurledJoin)
     (ReaderT EntityAlias FromIr)
     [(FieldName, FieldName)]
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames EntityAlias
joinAlias HashMap (Column 'BigQuery) (Column 'BigQuery)
HashMap ColumnName ColumnName
mapping)
      [(FieldName, FieldName)]
innerJoinFields <-
        ReaderT EntityAlias FromIr [(FieldName, FieldName)]
-> WriterT
     (Seq UnfurledJoin)
     (ReaderT EntityAlias FromIr)
     [(FieldName, FieldName)]
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames (From -> EntityAlias
fromAlias From
selectFrom) HashMap (Column 'BigQuery) (Column 'BigQuery)
HashMap ColumnName ColumnName
mapping)
      Expression
whereExpression <-
        ReaderT EntityAlias FromIr Expression
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) Expression
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift ((EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Expression
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const (From -> EntityAlias
fromAlias From
selectFrom)) (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
annBoolExp))
      Aggregate
aggregate <-
        ReaderT EntityAlias FromIr Aggregate
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) Aggregate
forall (m :: * -> *) a.
Monad m =>
m a -> WriterT (Seq UnfurledJoin) m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift
          ( (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr Aggregate
-> ReaderT EntityAlias FromIr Aggregate
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local
              (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const (From -> EntityAlias
fromAlias From
selectFrom))
              ( case AnnotatedAggregateOrderBy 'BigQuery Expression
annAggregateOrderBy of
                  AnnotatedAggregateOrderBy 'BigQuery Expression
Ir.AAOCount -> Aggregate -> ReaderT EntityAlias FromIr Aggregate
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Countable FieldName -> Aggregate
CountAggregate Countable FieldName
forall fieldname. Countable fieldname
StarCountable)
                  -- TODO(redactionExp): Deal with the redaction expression
                  Ir.AAOOp (Ir.AggregateOrderByColumn Text
text ColumnType 'BigQuery
_resultType ColumnInfo 'BigQuery
columnInfo AnnRedactionExp 'BigQuery Expression
_redactionExp) -> do
                    FieldName
fieldName <- ColumnInfo 'BigQuery -> ReaderT EntityAlias FromIr FieldName
fromColumnInfo ColumnInfo 'BigQuery
columnInfo
                    Aggregate -> ReaderT EntityAlias FromIr Aggregate
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Text -> Expression -> Aggregate
OpAggregate Text
text (FieldName -> Expression
ColumnExpression FieldName
fieldName))
              )
          )
      Seq UnfurledJoin
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) ()
forall w (m :: * -> *). MonadWriter w m => w -> m ()
tell
        ( UnfurledJoin -> Seq UnfurledJoin
forall a. a -> Seq a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
            ( UnfurledJoin
                { unfurledJoin :: Join
unfurledJoin =
                    Join
                      { $sel:joinSource:Join :: JoinSource
joinSource =
                          Select -> JoinSource
JoinSelect
                            Select
                              { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                                $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
                                $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
                                $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
                                $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
                                $sel:selectProjections:Select :: NonEmpty Projection
selectProjections =
                                  Aliased Aggregate -> Projection
AggregateProjection
                                    Aliased
                                      { $sel:aliasedThing:Aliased :: Aggregate
aliasedThing = Aggregate
aggregate,
                                        $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
alias
                                      }
                                    Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:|
                                    -- These are group by'd below in selectGroupBy.
                                    ((FieldName, FieldName) -> Projection)
-> [(FieldName, FieldName)] -> [Projection]
forall a b. (a -> b) -> [a] -> [b]
map
                                      ( \(FieldName
fieldName', FieldName
_) ->
                                          Aliased FieldName -> Projection
FieldNameProjection
                                            Aliased
                                              { $sel:aliasedThing:Aliased :: FieldName
aliasedThing = FieldName
fieldName',
                                                $sel:aliasedAlias:Aliased :: Text
aliasedAlias = FieldName -> Text
fieldName FieldName
fieldName'
                                              }
                                      )
                                      [(FieldName, FieldName)]
innerJoinFields,
                                From
$sel:selectFrom:Select :: From
selectFrom :: From
selectFrom,
                                $sel:selectJoins:Select :: [Join]
selectJoins = [],
                                $sel:selectWhere:Select :: Where
selectWhere = [Expression] -> Where
Where [Expression
whereExpression],
                                $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
                                $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing,
                                -- This group by corresponds to the field name projections above.
                                $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = ((FieldName, FieldName) -> FieldName)
-> [(FieldName, FieldName)] -> [FieldName]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, FieldName) -> FieldName
forall a b. (a, b) -> a
fst [(FieldName, FieldName)]
innerJoinFields
                              },
                        $sel:joinRightTable:Join :: EntityAlias
joinRightTable = From -> EntityAlias
fromAlias From
selectFrom,
                        $sel:joinProvenance:Join :: JoinProvenance
joinProvenance = JoinProvenance
OrderByJoinProvenance,
                        $sel:joinAlias:Join :: EntityAlias
joinAlias = EntityAlias
joinAlias,
                        [(FieldName, FieldName)]
$sel:joinOn:Join :: [(FieldName, FieldName)]
joinOn :: [(FieldName, FieldName)]
joinOn,
                        $sel:joinFieldName:Join :: Text
joinFieldName = TableName -> Text
tableNameText TableName 'BigQuery
TableName
tableName, -- TODO: not needed.
                        $sel:joinExtractPath:Join :: Maybe Text
joinExtractPath = Maybe Text
forall a. Maybe a
Nothing,
                        $sel:joinType:Join :: JoinType
joinType = JoinType
LeftOuter
                      },
                  unfurledObjectTableAlias :: Maybe (TableName, EntityAlias)
unfurledObjectTableAlias = Maybe (TableName, EntityAlias)
forall a. Maybe a
Nothing
                }
            )
        )
      FieldName
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall a.
a -> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        FieldName
          { $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = EntityAlias -> Text
entityAliasText EntityAlias
joinAlias,
            $sel:fieldName:FieldName :: Text
fieldName = Text
alias
          }
    Ir.AOCComputedField {} -> NonEmpty Error
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall a.
NonEmpty Error
-> WriterT (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (NonEmpty Error
 -> WriterT
      (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName)
-> NonEmpty Error
-> WriterT
     (Seq UnfurledJoin) (ReaderT EntityAlias FromIr) FieldName
forall a b. (a -> b) -> a -> b
$ Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
ComputedFieldsOrderByNotSupported

--------------------------------------------------------------------------------
-- Conversion functions

tableNameText :: TableName -> Text
tableNameText :: TableName -> Text
tableNameText (TableName {$sel:tableName:TableName :: TableName -> Text
tableName = Text
qname}) = Text
qname

-- | This is really the start where you query the base table,
-- everything else is joins attached to it.
fromQualifiedTable :: TableName -> FromIr From
fromQualifiedTable :: TableName -> FromIr From
fromQualifiedTable (TableName {$sel:tableNameSchema:TableName :: TableName -> Text
tableNameSchema = Text
schemaName, $sel:tableName:TableName :: TableName -> Text
tableName = Text
qname}) = do
  EntityAlias
alias <- NameTemplate -> FromIr EntityAlias
generateEntityAlias (Text -> NameTemplate
TableTemplate Text
qname)
  From -> FromIr From
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    ( Aliased TableName -> From
FromQualifiedTable
        ( Aliased
            { $sel:aliasedThing:Aliased :: TableName
aliasedThing =
                TableName {$sel:tableName:TableName :: Text
tableName = Text
qname, $sel:tableNameSchema:TableName :: Text
tableNameSchema = Text
schemaName},
              $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
entityAliasText EntityAlias
alias
            }
        )
    )

-- | Build a @'From' expression out of a function that returns a set of rows.
fromFunction ::
  -- | The parent's entity alias from which the column values for computed fields are referred
  ParentSelectFromEntity ->
  -- | The function
  FunctionName ->
  -- | List of positional Arguments
  [ArgumentExp Expression] ->
  -- | List of named arguments
  HashMap.HashMap Text (ArgumentExp Expression) ->
  FromIr From
fromFunction :: ParentSelectFromEntity
-> FunctionName
-> [ArgumentExp Expression]
-> HashMap Text (ArgumentExp Expression)
-> FromIr From
fromFunction ParentSelectFromEntity
parentEntityAlias FunctionName
functionName [ArgumentExp Expression]
positionalArgs HashMap Text (ArgumentExp Expression)
namedArgs = do
  EntityAlias
alias <- NameTemplate -> FromIr EntityAlias
generateEntityAlias (FunctionName -> NameTemplate
FunctionTemplate FunctionName
functionName)
  [Expression]
positionalArgExps <- (ArgumentExp Expression -> FromIr Expression)
-> [ArgumentExp Expression] -> FromIr [Expression]
forall (t :: * -> *) (m :: * -> *) a b.
(Traversable t, Monad m) =>
(a -> m b) -> t a -> m (t b)
forall (m :: * -> *) a b. Monad m => (a -> m b) -> [a] -> m [b]
mapM ArgumentExp Expression -> FromIr Expression
fromArgumentExp [ArgumentExp Expression]
positionalArgs
  [Expression]
namedArgExps <- [(Text, ArgumentExp Expression)]
-> ((Text, ArgumentExp Expression) -> FromIr Expression)
-> FromIr [Expression]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
t a -> (a -> f b) -> f (t b)
for (HashMap Text (ArgumentExp Expression)
-> [(Text, ArgumentExp Expression)]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList HashMap Text (ArgumentExp Expression)
namedArgs) (((Text, ArgumentExp Expression) -> FromIr Expression)
 -> FromIr [Expression])
-> ((Text, ArgumentExp Expression) -> FromIr Expression)
-> FromIr [Expression]
forall a b. (a -> b) -> a -> b
$ \(Text
argName, ArgumentExp Expression
argValue) -> Text -> Expression -> Expression
FunctionNamedArgument Text
argName (Expression -> Expression)
-> FromIr Expression -> FromIr Expression
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ArgumentExp Expression -> FromIr Expression
fromArgumentExp ArgumentExp Expression
argValue
  From -> FromIr From
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    ( Aliased SelectFromFunction -> From
FromFunction
        ( Aliased
            { $sel:aliasedThing:Aliased :: SelectFromFunction
aliasedThing = FunctionName -> [Expression] -> SelectFromFunction
SelectFromFunction FunctionName
functionName ([Expression]
positionalArgExps [Expression] -> [Expression] -> [Expression]
forall a. Semigroup a => a -> a -> a
<> [Expression]
namedArgExps),
              $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
entityAliasText EntityAlias
alias
            }
        )
    )
  where
    fromArgumentExp :: ArgumentExp Expression -> FromIr Expression
    fromArgumentExp :: ArgumentExp Expression -> FromIr Expression
fromArgumentExp = \case
      AEInput Expression
e -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Expression
e
      AETableColumn (ColumnName Text
columnName) -> do
        case ParentSelectFromEntity
parentEntityAlias of
          ParentSelectFromEntity
NoParentEntity -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (NonEmpty Error -> FromIr Expression)
-> NonEmpty Error -> FromIr Expression
forall a b. (a -> b) -> a -> b
$ Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
NoParentEntityInternalError
          ParentEntityAlias EntityAlias
entityAlias ->
            Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
              (Expression -> FromIr Expression)
-> Expression -> FromIr Expression
forall a b. (a -> b) -> a -> b
$ FieldName -> Expression
ColumnExpression
              (FieldName -> Expression) -> FieldName -> Expression
forall a b. (a -> b) -> a -> b
$ Text -> Text -> FieldName
FieldName Text
columnName (EntityAlias -> Text
entityAliasText EntityAlias
entityAlias)

fromAnnBoolExp ::
  Ir.GBoolExp 'BigQuery (Ir.AnnBoolExpFld 'BigQuery Expression) ->
  ReaderT EntityAlias FromIr Expression
fromAnnBoolExp :: GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp = (AnnBoolExpFld 'BigQuery Expression
 -> ReaderT EntityAlias FromIr Expression)
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr (GBoolExp 'BigQuery Expression)
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> GBoolExp 'BigQuery a -> f (GBoolExp 'BigQuery b)
traverse AnnBoolExpFld 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExpFld (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
 -> ReaderT EntityAlias FromIr (GBoolExp 'BigQuery Expression))
-> (GBoolExp 'BigQuery Expression
    -> ReaderT EntityAlias FromIr Expression)
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
forall (m :: * -> *) a b c.
Monad m =>
(a -> m b) -> (b -> m c) -> a -> m c
>=> GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp

fromNativeQuery :: NativeQuery 'BigQuery Expression -> FromIr From
fromNativeQuery :: NativeQuery 'BigQuery Expression -> FromIr From
fromNativeQuery NativeQuery {NativeQueryName
InterpolatedQuery Expression
LogicalModel 'BigQuery
nqRootFieldName :: NativeQueryName
nqInterpolatedQuery :: InterpolatedQuery Expression
nqLogicalModel :: LogicalModel 'BigQuery
nqRootFieldName :: forall (b :: BackendType) field.
NativeQuery b field -> NativeQueryName
nqInterpolatedQuery :: forall (b :: BackendType) field.
NativeQuery b field -> InterpolatedQuery field
nqLogicalModel :: forall (b :: BackendType) field.
NativeQuery b field -> LogicalModel b
..} = do
  EntityAlias
alias <- NameTemplate -> FromIr EntityAlias
generateEntityAlias (NativeQueryName -> NameTemplate
NativeQueryTemplate NativeQueryName
nqRootFieldName)
  let nqAlias :: Aliased NativeQueryName
nqAlias = NativeQueryName -> Text -> Aliased NativeQueryName
forall a. a -> Text -> Aliased a
Aliased NativeQueryName
nqRootFieldName (EntityAlias -> Text
entityAliasText EntityAlias
alias)
  FromIrWriter -> FromIr ()
forall w (m :: * -> *). MonadWriter w m => w -> m ()
tell (Map (Aliased NativeQueryName) (InterpolatedQuery Expression)
-> FromIrWriter
FromIrWriter (Aliased NativeQueryName
-> InterpolatedQuery Expression
-> Map (Aliased NativeQueryName) (InterpolatedQuery Expression)
forall k a. k -> a -> Map k a
M.singleton Aliased NativeQueryName
nqAlias InterpolatedQuery Expression
nqInterpolatedQuery))
  From -> FromIr From
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Aliased NativeQueryName -> From
FromNativeQuery Aliased NativeQueryName
nqAlias)

fromAnnBoolExpFld ::
  Ir.AnnBoolExpFld 'BigQuery Expression -> ReaderT EntityAlias FromIr Expression
fromAnnBoolExpFld :: AnnBoolExpFld 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExpFld =
  \case
    Ir.AVColumn ColumnInfo 'BigQuery
columnInfo [OpExpG 'BigQuery Expression]
opExpGs -> do
      Expression
expression <- (FieldName -> Expression)
-> ReaderT EntityAlias FromIr FieldName
-> ReaderT EntityAlias FromIr Expression
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap FieldName -> Expression
ColumnExpression (ColumnInfo 'BigQuery -> ReaderT EntityAlias FromIr FieldName
fromColumnInfo ColumnInfo 'BigQuery
columnInfo)
      [Expression]
expressions <- (OpExpG 'BigQuery Expression
 -> ReaderT EntityAlias FromIr Expression)
-> [OpExpG 'BigQuery Expression]
-> ReaderT EntityAlias FromIr [Expression]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse (FromIr Expression -> ReaderT EntityAlias FromIr Expression
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr Expression -> ReaderT EntityAlias FromIr Expression)
-> (OpExpG 'BigQuery Expression -> FromIr Expression)
-> OpExpG 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Expression -> OpExpG 'BigQuery Expression -> FromIr Expression
fromOpExpG Expression
expression) [OpExpG 'BigQuery Expression]
opExpGs
      Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ([Expression] -> Expression
AndExpression [Expression]
expressions)
    Ir.AVRelationship Rql.RelInfo {riTarget :: forall (b :: BackendType). RelInfo b -> RelTarget b
riTarget = Rql.RelTargetNativeQuery NativeQueryName
_} RelationshipFilters 'BigQuery Expression
_ ->
      [Char] -> ReaderT EntityAlias FromIr Expression
forall a. HasCallStack => [Char] -> a
error [Char]
"fromAnnBoolExpFld RelTargetNativeQuery"
    Ir.AVRelationship Rql.RelInfo {riMapping :: forall (b :: BackendType).
RelInfo b -> HashMap (Column b) (Column b)
riMapping = HashMap (Column 'BigQuery) (Column 'BigQuery)
mapping, riTarget :: forall (b :: BackendType). RelInfo b -> RelTarget b
riTarget = Rql.RelTargetTable TableName 'BigQuery
table} (Ir.RelationshipFilters GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
tablePerms GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
annBoolExp) -> do
      From
selectFrom <- FromIr From -> ReaderT EntityAlias FromIr From
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
table)
      [Expression]
foreignKeyConditions <- From
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [Expression]
fromMapping From
selectFrom HashMap (Column 'BigQuery) (Column 'BigQuery)
HashMap ColumnName ColumnName
mapping
      Expression
whereExpression <-
        (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Expression
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const (From -> EntityAlias
fromAlias From
selectFrom)) (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp ([GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)]
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
forall (backend :: BackendType) field.
[GBoolExp backend field] -> GBoolExp backend field
Ir.BoolAnd [GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
tablePerms, GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
annBoolExp]))
      Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( Select -> Expression
ExistsExpression
            Select
              { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
                $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
                $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
                $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
                $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
                $sel:selectProjections:Select :: NonEmpty Projection
selectProjections =
                  Aliased Expression -> Projection
ExpressionProjection
                    ( Aliased
                        { $sel:aliasedThing:Aliased :: Expression
aliasedThing = Expression
trueExpression,
                          $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
existsFieldName
                        }
                    )
                    Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| [],
                From
$sel:selectFrom:Select :: From
selectFrom :: From
selectFrom,
                $sel:selectJoins:Select :: [Join]
selectJoins = [Join]
forall a. Monoid a => a
mempty,
                $sel:selectWhere:Select :: Where
selectWhere = [Expression] -> Where
Where ([Expression]
foreignKeyConditions [Expression] -> [Expression] -> [Expression]
forall a. Semigroup a => a -> a -> a
<> [Expression
whereExpression]),
                $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
                $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing
              }
        )
    Ir.AVComputedField {} -> NonEmpty Error -> ReaderT EntityAlias FromIr Expression
forall a. NonEmpty Error -> ReaderT EntityAlias FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (NonEmpty Error -> ReaderT EntityAlias FromIr Expression)
-> NonEmpty Error -> ReaderT EntityAlias FromIr Expression
forall a b. (a -> b) -> a -> b
$ Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
ComputedFieldsBooleanExpressionNotSupported

fromColumnInfo :: Rql.ColumnInfo 'BigQuery -> ReaderT EntityAlias FromIr FieldName
fromColumnInfo :: ColumnInfo 'BigQuery -> ReaderT EntityAlias FromIr FieldName
fromColumnInfo Rql.ColumnInfo {ciColumn :: forall (b :: BackendType). ColumnInfo b -> Column b
ciColumn = ColumnName Text
column} = do
  EntityAlias {Text
$sel:entityAliasText:EntityAlias :: EntityAlias -> Text
entityAliasText :: Text
entityAliasText} <- ReaderT EntityAlias FromIr EntityAlias
forall r (m :: * -> *). MonadReader r m => m r
ask
  FieldName -> ReaderT EntityAlias FromIr FieldName
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    ( FieldName
        { $sel:fieldName:FieldName :: Text
fieldName = Text
column,
          $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = Text
entityAliasText
        }
    )

fromGExists :: Ir.GExists 'BigQuery Expression -> ReaderT EntityAlias FromIr Select
fromGExists :: GExists 'BigQuery Expression -> ReaderT EntityAlias FromIr Select
fromGExists Ir.GExists {TableName 'BigQuery
_geTable :: TableName 'BigQuery
_geTable :: forall (backend :: BackendType) field.
GExists backend field -> TableName backend
_geTable, GBoolExp 'BigQuery Expression
_geWhere :: GBoolExp 'BigQuery Expression
_geWhere :: forall (backend :: BackendType) field.
GExists backend field -> GBoolExp backend field
_geWhere} = do
  From
selectFrom <- FromIr From -> ReaderT EntityAlias FromIr From
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
_geTable)
  Expression
whereExpression <-
    (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Expression
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const (From -> EntityAlias
fromAlias From
selectFrom)) (GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp GBoolExp 'BigQuery Expression
_geWhere)
  Select -> ReaderT EntityAlias FromIr Select
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    Select
      { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
        $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
        $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
        $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
        $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
        $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
        $sel:selectProjections:Select :: NonEmpty Projection
selectProjections =
          Aliased Expression -> Projection
ExpressionProjection
            ( Aliased
                { $sel:aliasedThing:Aliased :: Expression
aliasedThing = Expression
trueExpression,
                  $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
existsFieldName
                }
            )
            Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| [],
        From
$sel:selectFrom:Select :: From
selectFrom :: From
selectFrom,
        $sel:selectJoins:Select :: [Join]
selectJoins = [Join]
forall a. Monoid a => a
mempty,
        $sel:selectWhere:Select :: Where
selectWhere = [Expression] -> Where
Where [Expression
whereExpression],
        $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
        $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing
      }

--------------------------------------------------------------------------------
-- Sources of projected fields
--
-- Because in the IR, a field projected can be a foreign object, we
-- have to both generate a projection AND on the side generate a join.
--
-- So a @FieldSource@ couples the idea of the projected thing and the
-- source of it (via 'Aliased').

data FieldSource
  = ExpressionFieldSource (Aliased Expression)
  | JoinFieldSource (Aliased Join)
  | AggregateFieldSource Text (NonEmpty (Aliased Aggregate))
  | ArrayAggFieldSource (Aliased ArrayAgg) (Maybe [FieldSource])
  deriving (FieldSource -> FieldSource -> Bool
(FieldSource -> FieldSource -> Bool)
-> (FieldSource -> FieldSource -> Bool) -> Eq FieldSource
forall a. (a -> a -> Bool) -> (a -> a -> Bool) -> Eq a
$c== :: FieldSource -> FieldSource -> Bool
== :: FieldSource -> FieldSource -> Bool
$c/= :: FieldSource -> FieldSource -> Bool
/= :: FieldSource -> FieldSource -> Bool
Eq, Int -> FieldSource -> ShowS
[FieldSource] -> ShowS
FieldSource -> [Char]
(Int -> FieldSource -> ShowS)
-> (FieldSource -> [Char])
-> ([FieldSource] -> ShowS)
-> Show FieldSource
forall a.
(Int -> a -> ShowS) -> (a -> [Char]) -> ([a] -> ShowS) -> Show a
$cshowsPrec :: Int -> FieldSource -> ShowS
showsPrec :: Int -> FieldSource -> ShowS
$cshow :: FieldSource -> [Char]
show :: FieldSource -> [Char]
$cshowList :: [FieldSource] -> ShowS
showList :: [FieldSource] -> ShowS
Show)

-- Example:
--
-- @
-- Track_aggregate {
--   aggregate {
--     count(columns: AlbumId)
--     foo: count(columns: AlbumId)
--     max {
--       AlbumId
--       TrackId
--     }
--   }
-- }
-- @
--
-- field =
-- @
-- TAFAgg
--   [ ( FieldName {getFieldNameTxt = "count"}
--     , AFCount (NonNullFieldCountable [ColumnName {columnName = "AlbumId"}]))
--   , ( FieldName {getFieldNameTxt = "foo"}
--     , AFCount (NonNullFieldCountable [ColumnName {columnName = "AlbumId"}]))
--   , ( FieldName {getFieldNameTxt = "max"}
--     , AFOp
--         (AggregateOp
--            { _aoOp = "max"
--            , _aoFields =
--                [ ( FieldName {getFieldNameTxt = "AlbumId"}
--                  , SFCol (ColumnName {columnName = "AlbumId"} (ColumnScalar IntegerScalarType)))
--                , ( FieldName {getFieldNameTxt = "TrackId"}
--                  , SFCol (ColumnName {columnName = "TrackId"} (ColumnScalar IntegerScalarType)))
--                ]
--            }))
--   ]
-- @
--
-- should produce:
--
-- SELECT COUNT(`t_Track1`.`AlbumId`) AS `count`,
--        COUNT(`t_Track1`.`AlbumId`) AS `foo`,
--        struct(max(`t_Track1`.`AlbumId`) AS `AlbumId`, max(`t_Track1`.`TrackId`) as TrackId) as `max`
-- FROM chinook.`Track` AS `t_Track1`
--
fromTableAggregateFieldG ::
  Args ->
  Top ->
  (Rql.FieldName, Ir.TableAggregateFieldG 'BigQuery Void Expression) ->
  ReaderT EntityAlias FromIr FieldSource
fromTableAggregateFieldG :: Args
-> Top
-> (FieldName, TableAggregateFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr FieldSource
fromTableAggregateFieldG Args
args Top
permissionBasedTop (Rql.FieldName Text
name, TableAggregateFieldG 'BigQuery Void Expression
field) =
  case TableAggregateFieldG 'BigQuery Void Expression
field of
    Ir.TAFAgg ([(FieldName, AggregateField 'BigQuery Expression)]
aggregateFields :: [(Rql.FieldName, Ir.AggregateField 'BigQuery Expression)]) ->
      case [(FieldName, AggregateField 'BigQuery Expression)]
-> Maybe
     (NonEmpty (FieldName, AggregateField 'BigQuery Expression))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty [(FieldName, AggregateField 'BigQuery Expression)]
aggregateFields of
        Maybe (NonEmpty (FieldName, AggregateField 'BigQuery Expression))
Nothing -> NonEmpty Error -> ReaderT EntityAlias FromIr FieldSource
forall a. NonEmpty Error -> ReaderT EntityAlias FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
NoAggregatesMustBeABug)
        Just NonEmpty (FieldName, AggregateField 'BigQuery Expression)
fields -> do
          NonEmpty (Aliased Aggregate)
aggregates <-
            ((FieldName, AggregateField 'BigQuery Expression)
 -> ReaderT EntityAlias FromIr (Aliased Aggregate))
-> NonEmpty (FieldName, AggregateField 'BigQuery Expression)
-> ReaderT EntityAlias FromIr (NonEmpty (Aliased Aggregate))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> NonEmpty a -> f (NonEmpty b)
traverse
              ( \(FieldName
fieldName, AggregateField 'BigQuery Expression
aggregateField) -> do
                  (Aggregate -> Aliased Aggregate)
-> ReaderT EntityAlias FromIr Aggregate
-> ReaderT EntityAlias FromIr (Aliased Aggregate)
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
                    ( \Aggregate
aliasedThing ->
                        Aliased {$sel:aliasedAlias:Aliased :: Text
aliasedAlias = FieldName -> Text
Rql.getFieldNameTxt FieldName
fieldName, Aggregate
$sel:aliasedThing:Aliased :: Aggregate
aliasedThing :: Aggregate
..}
                    )
                    (AggregateField 'BigQuery Expression
-> ReaderT EntityAlias FromIr Aggregate
fromAggregateField AggregateField 'BigQuery Expression
aggregateField)
              )
              NonEmpty (FieldName, AggregateField 'BigQuery Expression)
fields
          FieldSource -> ReaderT EntityAlias FromIr FieldSource
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Text -> NonEmpty (Aliased Aggregate) -> FieldSource
AggregateFieldSource Text
name NonEmpty (Aliased Aggregate)
aggregates)
    Ir.TAFExp Text
text ->
      FieldSource -> ReaderT EntityAlias FromIr FieldSource
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( Aliased Expression -> FieldSource
ExpressionFieldSource
            Aliased
              { $sel:aliasedThing:Aliased :: Expression
aliasedThing = TypedValue -> Expression
BigQuery.ValueExpression (ScalarType -> Value -> TypedValue
BigQuery.TypedValue ScalarType
BigQuery.StringScalarType (Text -> Value
StringValue Text
text)),
                $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name
              }
        )
    Ir.TAFNodes XNodesAgg 'BigQuery
_ (Fields (AnnFieldG 'BigQuery Void Expression)
fields :: [(Rql.FieldName, Ir.AnnFieldG 'BigQuery Void Expression)]) -> do
      [FieldSource]
fieldSources <-
        ((FieldName, AnnFieldG 'BigQuery Void Expression)
 -> ReaderT EntityAlias FromIr FieldSource)
-> Fields (AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr [FieldSource]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse
          (Map TableName EntityAlias
-> (FieldName, AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr FieldSource
fromAnnFieldsG (Args -> Map TableName EntityAlias
argsExistingJoins Args
args))
          Fields (AnnFieldG 'BigQuery Void Expression)
fields
      NonEmpty Projection
arrayAggProjections <- FromIr (NonEmpty Projection)
-> ReaderT EntityAlias FromIr (NonEmpty Projection)
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (Bool -> [FieldSource] -> FromIr (NonEmpty Projection)
selectProjectionsFromFieldSources Bool
False [FieldSource]
fieldSources)
      Top
globalTop <- FromIr Top -> ReaderT EntityAlias FromIr Top
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift FromIr Top
getGlobalTop
      let arrayAgg :: Aliased ArrayAgg
arrayAgg =
            Aliased
              { $sel:aliasedThing:Aliased :: ArrayAgg
aliasedThing =
                  ArrayAgg
                    { NonEmpty Projection
arrayAggProjections :: NonEmpty Projection
$sel:arrayAggProjections:ArrayAgg :: NonEmpty Projection
arrayAggProjections,
                      $sel:arrayAggOrderBy:ArrayAgg :: Maybe (NonEmpty OrderBy)
arrayAggOrderBy = Args -> Maybe (NonEmpty OrderBy)
argsOrderBy Args
args,
                      $sel:arrayAggTop:ArrayAgg :: Top
arrayAggTop = Top
globalTop Top -> Top -> Top
forall a. Semigroup a => a -> a -> a
<> Args -> Top
argsTop Args
args Top -> Top -> Top
forall a. Semigroup a => a -> a -> a
<> Top
permissionBasedTop
                    },
                $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name
              }
      FieldSource -> ReaderT EntityAlias FromIr FieldSource
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Aliased ArrayAgg -> Maybe [FieldSource] -> FieldSource
ArrayAggFieldSource Aliased ArrayAgg
arrayAgg ([FieldSource] -> Maybe [FieldSource]
forall a. a -> Maybe a
Just [FieldSource]
fieldSources))

fromAggregateField :: Ir.AggregateField 'BigQuery Expression -> ReaderT EntityAlias FromIr Aggregate
fromAggregateField :: AggregateField 'BigQuery Expression
-> ReaderT EntityAlias FromIr Aggregate
fromAggregateField AggregateField 'BigQuery Expression
aggregateField =
  case AggregateField 'BigQuery Expression
aggregateField of
    Ir.AFExp Text
text -> Aggregate -> ReaderT EntityAlias FromIr Aggregate
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Text -> Aggregate
TextAggregate Text
text)
    Ir.AFCount CountType 'BigQuery Expression
countType ->
      Countable FieldName -> Aggregate
CountAggregate (Countable FieldName -> Aggregate)
-> ReaderT EntityAlias FromIr (Countable FieldName)
-> ReaderT EntityAlias FromIr Aggregate
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> case Const (Countable ColumnName) Expression -> Countable ColumnName
forall {k} a (b :: k). Const a b -> a
getConst Const (Countable ColumnName) Expression
CountType 'BigQuery Expression
countType of
        Countable ColumnName
StarCountable -> Countable FieldName
-> ReaderT EntityAlias FromIr (Countable FieldName)
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Countable FieldName
forall fieldname. Countable fieldname
StarCountable
        NonNullFieldCountable NonEmpty ColumnName
names -> NonEmpty FieldName -> Countable FieldName
forall fieldname. NonEmpty fieldname -> Countable fieldname
NonNullFieldCountable (NonEmpty FieldName -> Countable FieldName)
-> ReaderT EntityAlias FromIr (NonEmpty FieldName)
-> ReaderT EntityAlias FromIr (Countable FieldName)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (ColumnName -> ReaderT EntityAlias FromIr FieldName)
-> NonEmpty ColumnName
-> ReaderT EntityAlias FromIr (NonEmpty FieldName)
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> NonEmpty a -> f (NonEmpty b)
traverse ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn NonEmpty ColumnName
names
        DistinctCountable NonEmpty ColumnName
names -> NonEmpty FieldName -> Countable FieldName
forall fieldname. NonEmpty fieldname -> Countable fieldname
DistinctCountable (NonEmpty FieldName -> Countable FieldName)
-> ReaderT EntityAlias FromIr (NonEmpty FieldName)
-> ReaderT EntityAlias FromIr (Countable FieldName)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> (ColumnName -> ReaderT EntityAlias FromIr FieldName)
-> NonEmpty ColumnName
-> ReaderT EntityAlias FromIr (NonEmpty FieldName)
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> NonEmpty a -> f (NonEmpty b)
traverse ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn NonEmpty ColumnName
names
    Ir.AFOp Ir.AggregateOp {$sel:_aoOp:AggregateOp :: forall (b :: BackendType) v. AggregateOp b v -> Text
_aoOp = Text
op, $sel:_aoFields:AggregateOp :: forall (b :: BackendType) v. AggregateOp b v -> SelectionFields b v
_aoFields = SelectionFields 'BigQuery Expression
fields} -> do
      NonEmpty (FieldName, SelectionField 'BigQuery Expression)
fs <- SelectionFields 'BigQuery Expression
-> Maybe
     (NonEmpty (FieldName, SelectionField 'BigQuery Expression))
forall a. [a] -> Maybe (NonEmpty a)
NE.nonEmpty SelectionFields 'BigQuery Expression
fields Maybe (NonEmpty (FieldName, SelectionField 'BigQuery Expression))
-> ReaderT
     EntityAlias
     FromIr
     (NonEmpty (FieldName, SelectionField 'BigQuery Expression))
-> ReaderT
     EntityAlias
     FromIr
     (NonEmpty (FieldName, SelectionField 'BigQuery Expression))
forall (m :: * -> *) a. Applicative m => Maybe a -> m a -> m a
`onNothing` NonEmpty Error
-> ReaderT
     EntityAlias
     FromIr
     (NonEmpty (FieldName, SelectionField 'BigQuery Expression))
forall a. NonEmpty Error -> ReaderT EntityAlias FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
MalformedAgg)
      NonEmpty (Text, Expression)
args <-
        ((FieldName, SelectionField 'BigQuery Expression)
 -> ReaderT EntityAlias FromIr (Text, Expression))
-> NonEmpty (FieldName, SelectionField 'BigQuery Expression)
-> ReaderT EntityAlias FromIr (NonEmpty (Text, Expression))
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> NonEmpty a -> f (NonEmpty b)
traverse
          ( \(Rql.FieldName Text
fieldName, SelectionField 'BigQuery Expression
columnField) -> do
              Expression
expression' <-
                case SelectionField 'BigQuery Expression
columnField of
                  Ir.SFCol Column 'BigQuery
column ColumnType 'BigQuery
_columnType AnnRedactionExp 'BigQuery Expression
_redactionExp -> (FieldName -> Expression)
-> ReaderT EntityAlias FromIr FieldName
-> ReaderT EntityAlias FromIr Expression
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap FieldName -> Expression
ColumnExpression (ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn Column 'BigQuery
ColumnName
column) -- TODO(redactionExp)
                  Ir.SFExp Text
text -> Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (TypedValue -> Expression
ValueExpression (ScalarType -> Value -> TypedValue
BigQuery.TypedValue ScalarType
BigQuery.StringScalarType (Text -> Value
StringValue Text
text)))
                  -- See Hasura.RQL.Types.Backend.supportsAggregateComputedFields
                  Ir.SFComputedField ComputedFieldName
_ ComputedFieldScalarSelect 'BigQuery Expression
_ -> [Char] -> ReaderT EntityAlias FromIr Expression
forall a. HasCallStack => [Char] -> a
error [Char]
"Aggregate computed fields aren't currently supported for BigQuery!"
              (Text, Expression) -> ReaderT EntityAlias FromIr (Text, Expression)
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Text
fieldName, Expression
expression')
          )
          NonEmpty (FieldName, SelectionField 'BigQuery Expression)
fs
      Aggregate -> ReaderT EntityAlias FromIr Aggregate
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Text -> NonEmpty (Text, Expression) -> Aggregate
OpAggregates Text
op NonEmpty (Text, Expression)
args)

-- | The main sources of fields, either constants, fields or via joins.
fromAnnFieldsG ::
  Map TableName EntityAlias ->
  (Rql.FieldName, Ir.AnnFieldG 'BigQuery Void Expression) ->
  ReaderT EntityAlias FromIr FieldSource
fromAnnFieldsG :: Map TableName EntityAlias
-> (FieldName, AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr FieldSource
fromAnnFieldsG Map TableName EntityAlias
existingJoins (Rql.FieldName Text
name, AnnFieldG 'BigQuery Void Expression
field) =
  case AnnFieldG 'BigQuery Void Expression
field of
    Ir.AFColumn AnnColumnField 'BigQuery Expression
annColumnField -> do
      Expression
expression <- AnnColumnField 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromAnnColumnField AnnColumnField 'BigQuery Expression
annColumnField
      FieldSource -> ReaderT EntityAlias FromIr FieldSource
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( Aliased Expression -> FieldSource
ExpressionFieldSource
            Aliased {$sel:aliasedThing:Aliased :: Expression
aliasedThing = Expression
expression, $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name}
        )
    Ir.AFExpression Text
text ->
      FieldSource -> ReaderT EntityAlias FromIr FieldSource
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( Aliased Expression -> FieldSource
ExpressionFieldSource
            Aliased
              { $sel:aliasedThing:Aliased :: Expression
aliasedThing = TypedValue -> Expression
BigQuery.ValueExpression (ScalarType -> Value -> TypedValue
BigQuery.TypedValue ScalarType
BigQuery.StringScalarType (Text -> Value
StringValue Text
text)),
                $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name
              }
        )
    Ir.AFObjectRelation ObjectRelationSelectG 'BigQuery Void Expression
objectRelationSelectG ->
      (Join -> FieldSource)
-> ReaderT EntityAlias FromIr Join
-> ReaderT EntityAlias FromIr FieldSource
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
        ( \Join
aliasedThing ->
            Aliased Join -> FieldSource
JoinFieldSource (Aliased {Join
$sel:aliasedThing:Aliased :: Join
aliasedThing :: Join
aliasedThing, $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name})
        )
        (Map TableName EntityAlias
-> ObjectRelationSelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromObjectRelationSelectG Map TableName EntityAlias
existingJoins ObjectRelationSelectG 'BigQuery Void Expression
objectRelationSelectG)
    Ir.AFArrayRelation ArraySelectG 'BigQuery Void Expression
arraySelectG ->
      (Join -> FieldSource)
-> ReaderT EntityAlias FromIr Join
-> ReaderT EntityAlias FromIr FieldSource
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
        ( \Join
aliasedThing ->
            Aliased Join -> FieldSource
JoinFieldSource (Aliased {Join
$sel:aliasedThing:Aliased :: Join
aliasedThing :: Join
aliasedThing, $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name})
        )
        (ArraySelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromArraySelectG ArraySelectG 'BigQuery Void Expression
arraySelectG)
    Ir.AFComputedField XComputedField 'BigQuery
_ ComputedFieldName
_ ComputedFieldSelect 'BigQuery Void Expression
computedFieldSelect -> do
      Expression
expression <- ComputedFieldSelect 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Expression
fromComputedFieldSelect ComputedFieldSelect 'BigQuery Void Expression
computedFieldSelect
      FieldSource -> ReaderT EntityAlias FromIr FieldSource
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( Aliased Expression -> FieldSource
ExpressionFieldSource
            Aliased {$sel:aliasedThing:Aliased :: Expression
aliasedThing = Expression
expression, $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name}
        )

-- | Here is where we project a field as a column expression. If
-- number stringification is on, then we wrap it in a
-- 'ToStringExpression' so that it's casted when being projected.
fromAnnColumnField ::
  Ir.AnnColumnField 'BigQuery Expression ->
  ReaderT EntityAlias FromIr Expression
fromAnnColumnField :: AnnColumnField 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromAnnColumnField AnnColumnField 'BigQuery Expression
annColumnField = do
  FieldName
fieldName <- ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn Column 'BigQuery
ColumnName
column
  if Bool
asText Bool -> Bool -> Bool
|| Bool
False -- TODO: (Rql.isScalarColumnWhere Psql.isBigNum typ && stringifyNumbers == Rql.StringifyNumbers)
    then Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression
ToStringExpression (FieldName -> Expression
ColumnExpression FieldName
fieldName))
    else case AnnRedactionExp 'BigQuery Expression
redactionExp of
      AnnRedactionExp 'BigQuery Expression
Ir.NoRedaction -> Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (FieldName -> Expression
ColumnExpression FieldName
fieldName)
      Ir.RedactIfFalse GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
ex -> do
        Expression
ex' <- ((AnnBoolExpFld 'BigQuery Expression
 -> ReaderT EntityAlias FromIr Expression)
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr (GBoolExp 'BigQuery Expression)
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> GBoolExp 'BigQuery a -> f (GBoolExp 'BigQuery b)
traverse AnnBoolExpFld 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExpFld (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
 -> ReaderT EntityAlias FromIr (GBoolExp 'BigQuery Expression))
-> (GBoolExp 'BigQuery Expression
    -> ReaderT EntityAlias FromIr Expression)
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
forall (m :: * -> *) a b c.
Monad m =>
(a -> m b) -> (b -> m c) -> a -> m c
>=> GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp) (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
forall a b. Coercible a b => a -> b
coerce GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
ex)
        Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> FieldName -> Expression
ConditionalProjection Expression
ex' FieldName
fieldName)
  where
    Ir.AnnColumnField
      { $sel:_acfColumn:AnnColumnField :: forall (b :: BackendType) v. AnnColumnField b v -> Column b
_acfColumn = Column 'BigQuery
column,
        $sel:_acfAsText:AnnColumnField :: forall (b :: BackendType) v. AnnColumnField b v -> Bool
_acfAsText = Bool
asText :: Bool,
        $sel:_acfArguments:AnnColumnField :: forall (b :: BackendType) v.
AnnColumnField b v -> Maybe (ScalarSelectionArguments b)
_acfArguments = Maybe Void
_ :: Maybe Void,
        $sel:_acfRedactionExpression:AnnColumnField :: forall (b :: BackendType) v.
AnnColumnField b v -> AnnRedactionExp b v
_acfRedactionExpression = AnnRedactionExp 'BigQuery Expression
redactionExp :: Ir.AnnRedactionExp 'BigQuery Expression
      } = AnnColumnField 'BigQuery Expression
annColumnField

-- | This is where a field name "foo" is resolved to a fully qualified
-- field name [table].[foo]. The table name comes from EntityAlias in
-- the ReaderT.
fromColumn :: ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn :: ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn (ColumnName Text
txt) = do
  EntityAlias {Text
$sel:entityAliasText:EntityAlias :: EntityAlias -> Text
entityAliasText :: Text
entityAliasText} <- ReaderT EntityAlias FromIr EntityAlias
forall r (m :: * -> *). MonadReader r m => m r
ask
  FieldName -> ReaderT EntityAlias FromIr FieldName
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (FieldName {$sel:fieldName:FieldName :: Text
fieldName = Text
txt, $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = Text
entityAliasText})

fieldSourceProjections :: Bool -> FieldSource -> FromIr (NonEmpty Projection)
fieldSourceProjections :: Bool -> FieldSource -> FromIr (NonEmpty Projection)
fieldSourceProjections Bool
keepJoinField =
  \case
    ExpressionFieldSource Aliased Expression
aliasedExpression ->
      NonEmpty Projection -> FromIr (NonEmpty Projection)
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Aliased Expression -> Projection
ExpressionProjection Aliased Expression
aliasedExpression Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| [])
    JoinFieldSource Aliased Join
aliasedJoin ->
      [Projection] -> FromIr (NonEmpty Projection)
forall (m :: * -> *) x.
MonadValidate (NonEmpty Error) m =>
[x] -> m (NonEmpty x)
toNonEmpty
        -- Here we're producing all join fields needed later for
        -- Haskell-native joining.  They will be removed by upstream
        -- code if keepJoinField is True
        ( [ Aliased FieldName -> Projection
FieldNameProjection
              ( Aliased
                  { $sel:aliasedThing:Aliased :: FieldName
aliasedThing = FieldName
right,
                    $sel:aliasedAlias:Aliased :: Text
aliasedAlias = FieldName -> Text
fieldNameText FieldName
right
                  }
              )
            | Bool
keepJoinField,
              (FieldName
_left, FieldName
right) <- Join -> [(FieldName, FieldName)]
joinOn Join
join'
          ]
            [Projection] -> [Projection] -> [Projection]
forall a. Semigroup a => a -> a -> a
<>
            -- Below:
            -- When we're doing an array-aggregate, e.g.
            --
            -- query MyQuery {
            --   hasura_Artist {
            --     albums_aggregate {
            --       aggregate {
            --         count
            --       }
            --     }
            --   }
            -- }
            --
            -- we're going to do a join on the albums table, and that
            -- join query will produce a single-row result. Therefore we
            -- can grab the whole entity as a STRUCT-typed object. See
            -- also the docs for 'fromArrayRelationSelectG' and for
            -- 'fromArrayAggregateSelectG'.
            case Join -> JoinProvenance
joinProvenance Join
join' of
              ArrayJoinProvenance [Text]
fields ->
                Projection -> [Projection]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure
                  ( EntityAlias -> Aliased [FieldName] -> Projection
ArrayEntityProjection
                      (Join -> EntityAlias
joinAlias Join
join')
                      Aliased Join
aliasedJoin
                        { $sel:aliasedThing:Aliased :: [FieldName]
aliasedThing =
                            (Text -> FieldName) -> [Text] -> [FieldName]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
                              ( \Text
name ->
                                  FieldName
                                    { $sel:fieldName:FieldName :: Text
fieldName = Text
name,
                                      $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity =
                                        EntityAlias -> Text
entityAliasText (Join -> EntityAlias
joinAlias Join
join')
                                    }
                              )
                              [Text]
fields,
                          $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Aliased Join -> Text
forall a. Aliased a -> Text
aliasedAlias Aliased Join
aliasedJoin
                        }
                  )
              ObjectJoinProvenance [Text]
fields ->
                Projection -> [Projection]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure
                  ( Aliased [(FieldName, FieldOrigin)] -> Projection
EntityProjection
                      Aliased Join
aliasedJoin
                        { $sel:aliasedThing:Aliased :: [(FieldName, FieldOrigin)]
aliasedThing =
                            (Text -> (FieldName, FieldOrigin))
-> [Text] -> [(FieldName, FieldOrigin)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
                              ( \Text
name ->
                                  ( FieldName
                                      { $sel:fieldName:FieldName :: Text
fieldName = Text
name,
                                        $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity =
                                          EntityAlias -> Text
entityAliasText (Join -> EntityAlias
joinAlias Join
join')
                                      },
                                    FieldOrigin
NoOrigin
                                  )
                              )
                              [Text]
fields,
                          $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Aliased Join -> Text
forall a. Aliased a -> Text
aliasedAlias Aliased Join
aliasedJoin
                        }
                  )
              ArrayAggregateJoinProvenance [(Text, FieldOrigin)]
fields ->
                Projection -> [Projection]
forall a. a -> [a]
forall (f :: * -> *) a. Applicative f => a -> f a
pure
                  ( Aliased [(FieldName, FieldOrigin)] -> Projection
EntityProjection
                      Aliased Join
aliasedJoin
                        { $sel:aliasedThing:Aliased :: [(FieldName, FieldOrigin)]
aliasedThing =
                            ((Text, FieldOrigin) -> (FieldName, FieldOrigin))
-> [(Text, FieldOrigin)] -> [(FieldName, FieldOrigin)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
                              ( \(Text
name, FieldOrigin
fieldOrigin) ->
                                  ( FieldName
                                      { $sel:fieldName:FieldName :: Text
fieldName = Text
name,
                                        $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity =
                                          EntityAlias -> Text
entityAliasText (Join -> EntityAlias
joinAlias Join
join')
                                      },
                                    FieldOrigin
fieldOrigin
                                  )
                              )
                              [(Text, FieldOrigin)]
fields,
                          $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Aliased Join -> Text
forall a. Aliased a -> Text
aliasedAlias Aliased Join
aliasedJoin
                        }
                  )
              JoinProvenance
_ -> []
        )
      where
        join' :: Join
join' = Aliased Join -> Join
forall a. Aliased a -> a
aliasedThing Aliased Join
aliasedJoin
    AggregateFieldSource Text
name NonEmpty (Aliased Aggregate)
aggregates ->
      NonEmpty Projection -> FromIr (NonEmpty Projection)
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
        ( Aliased (NonEmpty (Aliased Aggregate)) -> Projection
AggregateProjections
            (Aliased {$sel:aliasedThing:Aliased :: NonEmpty (Aliased Aggregate)
aliasedThing = NonEmpty (Aliased Aggregate)
aggregates, $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name})
            Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| []
        )
    ArrayAggFieldSource Aliased ArrayAgg
arrayAgg Maybe [FieldSource]
_ -> NonEmpty Projection -> FromIr (NonEmpty Projection)
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Aliased ArrayAgg -> Projection
ArrayAggProjection Aliased ArrayAgg
arrayAgg Projection -> [Projection] -> NonEmpty Projection
forall a. a -> [a] -> NonEmpty a
:| [])
  where
    fieldNameText :: FieldName -> Text
fieldNameText FieldName {Text
$sel:fieldName:FieldName :: FieldName -> Text
fieldName :: Text
fieldName} = Text
fieldName

fieldSourceJoins :: FieldSource -> Maybe [Join]
fieldSourceJoins :: FieldSource -> Maybe [Join]
fieldSourceJoins =
  \case
    JoinFieldSource Aliased Join
aliasedJoin -> [Join] -> Maybe [Join]
forall a. a -> Maybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure [Aliased Join -> Join
forall a. Aliased a -> a
aliasedThing Aliased Join
aliasedJoin]
    ExpressionFieldSource {} -> Maybe [Join]
forall a. Maybe a
Nothing
    AggregateFieldSource {} -> Maybe [Join]
forall a. Maybe a
Nothing
    ArrayAggFieldSource Aliased ArrayAgg
_ Maybe [FieldSource]
sources -> ([FieldSource] -> [Join]) -> Maybe [FieldSource] -> Maybe [Join]
forall a b. (a -> b) -> Maybe a -> Maybe b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap ([[Join]] -> [Join]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
concat ([[Join]] -> [Join])
-> ([FieldSource] -> [[Join]]) -> [FieldSource] -> [Join]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (FieldSource -> Maybe [Join]) -> [FieldSource] -> [[Join]]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe FieldSource -> Maybe [Join]
fieldSourceJoins) Maybe [FieldSource]
sources

--------------------------------------------------------------------------------
-- Joins

-- | Produce the join for an object relation. We produce a normal
-- select, but then include join fields. Then downstream, the
-- DataLoader will execute the lhs select and rhs join in separate
-- server queries, then do a Haskell-native join on the join fields.
--
-- See also 'fromArrayRelationSelectG' for similar example.
fromObjectRelationSelectG ::
  Map TableName EntityAlias ->
  Ir.ObjectRelationSelectG 'BigQuery Void Expression ->
  ReaderT EntityAlias FromIr Join
-- We're not using existingJoins at the moment, which was used to
-- avoid re-joining on the same table twice.
fromObjectRelationSelectG :: Map TableName EntityAlias
-> ObjectRelationSelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromObjectRelationSelectG Map TableName EntityAlias
_existingJoins ObjectRelationSelectG 'BigQuery Void Expression
annRelationSelectG = do
  From
selectFrom <- case SelectFromG 'BigQuery Expression
target of
    Ir.FromTable TableName 'BigQuery
t -> FromIr From -> ReaderT EntityAlias FromIr From
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr From -> ReaderT EntityAlias FromIr From)
-> FromIr From -> ReaderT EntityAlias FromIr From
forall a b. (a -> b) -> a -> b
$ TableName -> FromIr From
fromQualifiedTable TableName 'BigQuery
TableName
t
    Ir.FromNativeQuery NativeQuery 'BigQuery Expression
nq -> FromIr From -> ReaderT EntityAlias FromIr From
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr From -> ReaderT EntityAlias FromIr From)
-> FromIr From -> ReaderT EntityAlias FromIr From
forall a b. (a -> b) -> a -> b
$ NativeQuery 'BigQuery Expression -> FromIr From
fromNativeQuery NativeQuery 'BigQuery Expression
nq
    SelectFromG 'BigQuery Expression
other -> [Char] -> ReaderT EntityAlias FromIr From
forall a. HasCallStack => [Char] -> a
error ([Char] -> ReaderT EntityAlias FromIr From)
-> [Char] -> ReaderT EntityAlias FromIr From
forall a b. (a -> b) -> a -> b
$ [Char]
"fromObjectRelationSelectG: " [Char] -> ShowS
forall a. Semigroup a => a -> a -> a
<> SelectFromG 'BigQuery Expression -> [Char]
forall a. Show a => a -> [Char]
show SelectFromG 'BigQuery Expression
other
  let EntityAlias
entityAlias :: EntityAlias = From -> EntityAlias
fromAlias From
selectFrom
  [FieldSource]
fieldSources <-
    (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr [FieldSource]
-> ReaderT EntityAlias FromIr [FieldSource]
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local
      (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const EntityAlias
entityAlias)
      (((FieldName, AnnFieldG 'BigQuery Void Expression)
 -> ReaderT EntityAlias FromIr FieldSource)
-> Fields (AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr [FieldSource]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse (Map TableName EntityAlias
-> (FieldName, AnnFieldG 'BigQuery Void Expression)
-> ReaderT EntityAlias FromIr FieldSource
fromAnnFieldsG Map TableName EntityAlias
forall a. Monoid a => a
mempty) Fields (AnnFieldG 'BigQuery Void Expression)
fields)
  NonEmpty Projection
selectProjections <- FromIr (NonEmpty Projection)
-> ReaderT EntityAlias FromIr (NonEmpty Projection)
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (Bool -> [FieldSource] -> FromIr (NonEmpty Projection)
selectProjectionsFromFieldSources Bool
True [FieldSource]
fieldSources)
  Text
joinFieldName <- FromIr Text -> ReaderT EntityAlias FromIr Text
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (RelName -> FromIr Text
fromRelName RelName
_aarRelationshipName)
  EntityAlias
joinAlias <-
    FromIr EntityAlias -> ReaderT EntityAlias FromIr EntityAlias
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (NameTemplate -> FromIr EntityAlias
generateEntityAlias (Text -> NameTemplate
ObjectRelationTemplate Text
joinFieldName))
  Expression
filterExpression <- (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Expression
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const EntityAlias
entityAlias) (GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
-> ReaderT EntityAlias FromIr Expression
fromAnnBoolExp GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
tableFilter)

  -- @mapping@ here describes the pairs of columns that form foreign key
  -- relationships between tables. For example, when querying an "article"
  -- table for article titles and joining on an "authors" table for author
  -- names, we could end up with something like the following:
  --
  -- @
  -- [ ( ColumnName { columnName = "author_id" }
  --   , ColumnName { columnName = "id" }
  --   )
  -- ]
  -- @
  --
  -- Note that the "local" table is on the left, and the "remote" table is on
  -- the right.

  [(FieldName, FieldName)]
joinFields <- EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames (From -> EntityAlias
fromAlias From
selectFrom) HashMap ColumnName ColumnName
mapping
  [(FieldName, FieldName)]
joinOn <- EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames EntityAlias
joinAlias HashMap ColumnName ColumnName
mapping
  NonEmpty Projection
joinFieldProjections <- [Projection] -> ReaderT EntityAlias FromIr (NonEmpty Projection)
forall (m :: * -> *) x.
MonadValidate (NonEmpty Error) m =>
[x] -> m (NonEmpty x)
toNonEmpty (((FieldName, FieldName) -> Projection)
-> [(FieldName, FieldName)] -> [Projection]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, FieldName) -> Projection
prepareJoinFieldProjection [(FieldName, FieldName)]
joinFields)

  let selectFinalWantedFields :: Maybe [Text]
selectFinalWantedFields = [Text] -> Maybe [Text]
forall a. a -> Maybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Fields (AnnFieldG 'BigQuery Void Expression) -> [Text]
fieldTextNames Fields (AnnFieldG 'BigQuery Void Expression)
fields)
  Join -> ReaderT EntityAlias FromIr Join
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    Join
      { EntityAlias
$sel:joinAlias:Join :: EntityAlias
joinAlias :: EntityAlias
joinAlias,
        $sel:joinSource:Join :: JoinSource
joinSource =
          Select -> JoinSource
JoinSelect
            Select
              { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
                $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
                Maybe [Text]
$sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields :: Maybe [Text]
selectFinalWantedFields,
                $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty,
                $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
                $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
                $sel:selectProjections:Select :: NonEmpty Projection
selectProjections = NonEmpty Projection
joinFieldProjections NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection
forall a. Semigroup a => a -> a -> a
<> NonEmpty Projection
selectProjections,
                From
$sel:selectFrom:Select :: From
selectFrom :: From
selectFrom,
                $sel:selectJoins:Select :: [Join]
selectJoins = [[Join]] -> [Join]
forall (t :: * -> *) a. Foldable t => t [a] -> [a]
concat ((FieldSource -> Maybe [Join]) -> [FieldSource] -> [[Join]]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe FieldSource -> Maybe [Join]
fieldSourceJoins [FieldSource]
fieldSources),
                $sel:selectWhere:Select :: Where
selectWhere = [Expression] -> Where
Where [Expression
filterExpression],
                $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing
              },
        [(FieldName, FieldName)]
$sel:joinOn:Join :: [(FieldName, FieldName)]
joinOn :: [(FieldName, FieldName)]
joinOn,
        $sel:joinRightTable:Join :: EntityAlias
joinRightTable = From -> EntityAlias
fromAlias From
selectFrom,
        $sel:joinProvenance:Join :: JoinProvenance
joinProvenance =
          [Text] -> JoinProvenance
ObjectJoinProvenance
            ([Text] -> Maybe [Text] -> [Text]
forall a. a -> Maybe a -> a
fromMaybe [] Maybe [Text]
selectFinalWantedFields), -- TODO: OK?
            -- Above: Needed by DataLoader to determine the type of
            -- Haskell-native join to perform.
        Text
$sel:joinFieldName:Join :: Text
joinFieldName :: Text
joinFieldName,
        $sel:joinExtractPath:Join :: Maybe Text
joinExtractPath = Maybe Text
forall a. Maybe a
Nothing,
        $sel:joinType:Join :: JoinType
joinType = case Nullable
nullable of Nullable
Nullable -> JoinType
LeftOuter; Nullable
NotNullable -> JoinType
Inner
      }
  where
    Ir.AnnObjectSelectG
      { $sel:_aosFields:AnnObjectSelectG :: forall (b :: BackendType) r v.
AnnObjectSelectG b r v -> AnnFieldsG b r v
_aosFields = Fields (AnnFieldG 'BigQuery Void Expression)
fields :: Ir.AnnFieldsG 'BigQuery Void Expression,
        $sel:_aosTarget:AnnObjectSelectG :: forall (b :: BackendType) r v.
AnnObjectSelectG b r v -> SelectFromG b v
_aosTarget = SelectFromG 'BigQuery Expression
target :: Ir.SelectFromG 'BigQuery Expression,
        $sel:_aosTargetFilter:AnnObjectSelectG :: forall (b :: BackendType) r v.
AnnObjectSelectG b r v -> AnnBoolExp b v
_aosTargetFilter = GBoolExp 'BigQuery (AnnBoolExpFld 'BigQuery Expression)
tableFilter :: Ir.AnnBoolExp 'BigQuery Expression
      } = AnnObjectSelectG 'BigQuery Void Expression
annObjectSelectG
    Ir.AnnRelationSelectG
      { RelName
_aarRelationshipName :: RelName
$sel:_aarRelationshipName:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> RelName
_aarRelationshipName,
        $sel:_aarColumnMapping:AnnRelationSelectG :: forall (b :: BackendType) a.
AnnRelationSelectG b a -> HashMap (Column b) (Column b)
_aarColumnMapping = HashMap ColumnName ColumnName
mapping :: HashMap ColumnName ColumnName,
        $sel:_aarAnnSelect:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> a
_aarAnnSelect = AnnObjectSelectG 'BigQuery Void Expression
annObjectSelectG :: Ir.AnnObjectSelectG 'BigQuery Void Expression,
        $sel:_aarNullable:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> Nullable
_aarNullable = Nullable
nullable
      } = ObjectRelationSelectG 'BigQuery Void Expression
annRelationSelectG

-- We're not using existingJoins at the moment, which was used to
-- avoid re-joining on the same table twice.
_lookupTableFrom ::
  Map TableName EntityAlias ->
  TableName ->
  FromIr (Either EntityAlias From)
_lookupTableFrom :: Map TableName EntityAlias
-> TableName -> FromIr (Either EntityAlias From)
_lookupTableFrom Map TableName EntityAlias
existingJoins TableName
tableFrom = do
  case TableName -> Map TableName EntityAlias -> Maybe EntityAlias
forall k a. Ord k => k -> Map k a -> Maybe a
M.lookup TableName
tableFrom Map TableName EntityAlias
existingJoins of
    Just EntityAlias
entityAlias -> Either EntityAlias From -> FromIr (Either EntityAlias From)
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (EntityAlias -> Either EntityAlias From
forall a b. a -> Either a b
Left EntityAlias
entityAlias)
    Maybe EntityAlias
Nothing -> (From -> Either EntityAlias From)
-> FromIr From -> FromIr (Either EntityAlias From)
forall a b. (a -> b) -> FromIr a -> FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap From -> Either EntityAlias From
forall a b. b -> Either a b
Right (TableName -> FromIr From
fromQualifiedTable TableName
tableFrom)

fromArraySelectG :: Ir.ArraySelectG 'BigQuery Void Expression -> ReaderT EntityAlias FromIr Join
fromArraySelectG :: ArraySelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromArraySelectG =
  \case
    Ir.ASSimple ArrayRelationSelectG 'BigQuery Void Expression
arrayRelationSelectG ->
      ArrayRelationSelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromArrayRelationSelectG ArrayRelationSelectG 'BigQuery Void Expression
arrayRelationSelectG
    Ir.ASAggregate ArrayAggregateSelectG 'BigQuery Void Expression
arrayAggregateSelectG ->
      ArrayAggregateSelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromArrayAggregateSelectG ArrayAggregateSelectG 'BigQuery Void Expression
arrayAggregateSelectG

-- | Generate a select field @'Expression' for a computed field
--
-- > ARRAY(
-- >   SELECT
-- >     AS STRUCT
-- >     `column_1`,
-- >     `column_2`,
-- >     `column_3`
-- >   FROM
-- >     UNNEST(
-- >       ARRAY(
-- >           SELECT AS STRUCT *
-- >           FROM `dataset`.`function_name`(`argument_name` => `parent_entity`.`column`)
-- >       )
-- >     )
-- >   LIMIT 1000 -- global limit
-- > ) AS `field_name`
--
-- Using 'LIMIT' right after 'FROM <function>' expression raises query exception.
-- To avoid this problem, we are packing and unpacking the rows returned from the function
-- using 'ARRAY' and 'UNNEST', then applying LIMIT. Somehow this is working with exact reason
-- being unknown. See https://github.com/hasura/graphql-engine/issues/8562 for more details.
fromComputedFieldSelect ::
  Ir.ComputedFieldSelect 'BigQuery Void Expression ->
  ReaderT EntityAlias FromIr Expression
fromComputedFieldSelect :: ComputedFieldSelect 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Expression
fromComputedFieldSelect = \case
  Ir.CFSScalar {} ->
    -- As of now, we don't have support for computed fields returning a scalar value.
    -- See https://github.com/hasura/graphql-engine/issues/8521
    NonEmpty Error -> ReaderT EntityAlias FromIr Expression
forall a. NonEmpty Error -> ReaderT EntityAlias FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (NonEmpty Error -> ReaderT EntityAlias FromIr Expression)
-> NonEmpty Error -> ReaderT EntityAlias FromIr Expression
forall a b. (a -> b) -> a -> b
$ Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
ScalarComputedFieldsNotSupported
  Ir.CFSTable JsonAggSelect
jsonAggSelect AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSimpleSelect -> do
    EntityAlias
entityAlias <- ReaderT EntityAlias FromIr EntityAlias
forall r (m :: * -> *). MonadReader r m => m r
ask
    Select
select <- FromIr Select -> ReaderT EntityAlias FromIr Select
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr Select -> ReaderT EntityAlias FromIr Select)
-> FromIr Select -> ReaderT EntityAlias FromIr Select
forall a b. (a -> b) -> a -> b
$ PartitionableSelect -> Select
noExtraPartitionFields (PartitionableSelect -> Select)
-> FromIr PartitionableSelect -> FromIr Select
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> ParentSelectFromEntity
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr PartitionableSelect
fromSelectRows (EntityAlias -> ParentSelectFromEntity
ParentEntityAlias EntityAlias
entityAlias) AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSimpleSelect
    let selectWithCardinality :: Select
selectWithCardinality =
          Select
select
            { $sel:selectCardinality:Select :: Cardinality
selectCardinality =
                case JsonAggSelect
jsonAggSelect of
                  JsonAggSelect
Rql.JASMultipleRows -> Cardinality
Many
                  JsonAggSelect
Rql.JASSingleObject -> Cardinality
One,
              $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
AsStruct,
              $sel:selectFrom:Select :: From
selectFrom = From -> From
wrapUnnest (Select -> From
selectFrom Select
select)
            }
    Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> ReaderT EntityAlias FromIr Expression)
-> Expression -> ReaderT EntityAlias FromIr Expression
forall a b. (a -> b) -> a -> b
$ Select -> Expression
applyArrayOnSelect Select
selectWithCardinality
  where
    applyArrayOnSelect :: Select -> Expression
    applyArrayOnSelect :: Select -> Expression
applyArrayOnSelect Select
select =
      FunctionName -> [Expression] -> Expression
FunctionExpression (Text -> Maybe Text -> FunctionName
FunctionName Text
"ARRAY" Maybe Text
forall a. Maybe a
Nothing) [Select -> Expression
SelectExpression Select
select]

    wrapUnnest :: From -> From
    wrapUnnest :: From -> From
wrapUnnest From
from =
      let starSelect :: Select
starSelect =
            Select
              { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
                $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
AsStruct,
                $sel:selectProjections:Select :: NonEmpty Projection
selectProjections = Projection -> NonEmpty Projection
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Projection
StarProjection,
                $sel:selectFrom:Select :: From
selectFrom = From
from,
                $sel:selectJoins:Select :: [Join]
selectJoins = [],
                $sel:selectWhere:Select :: Where
selectWhere = [Expression] -> Where
Where [],
                $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing,
                $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing,
                $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [],
                $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Maybe [Text]
forall a. Maybe a
Nothing,
                $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
Many
              }
          arraySelect :: Expression
arraySelect = Select -> Expression
applyArrayOnSelect Select
starSelect
       in Aliased SelectFromFunction -> From
FromFunction
            Aliased
              { $sel:aliasedThing:Aliased :: SelectFromFunction
aliasedThing = FunctionName -> [Expression] -> SelectFromFunction
SelectFromFunction (Text -> Maybe Text -> FunctionName
FunctionName Text
"UNNEST" Maybe Text
forall a. Maybe a
Nothing) [Expression
arraySelect],
                $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
entityAliasText (From -> EntityAlias
fromAlias From
from)
              }

-- | Produce the join for an array aggregate relation. We produce a
-- normal select, but then include join fields. Then downstream, the
-- DataLoader will execute the lhs select and rhs join in separate
-- server queries, then do a Haskell-native join on the join fields.
--
-- See also 'fromArrayRelationSelectG' for similar example.
fromArrayAggregateSelectG ::
  Ir.AnnRelationSelectG 'BigQuery (Ir.AnnAggregateSelectG 'BigQuery Void Expression) ->
  ReaderT EntityAlias FromIr Join
fromArrayAggregateSelectG :: ArrayAggregateSelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromArrayAggregateSelectG ArrayAggregateSelectG 'BigQuery Void Expression
annRelationSelectG = do
  Text
joinFieldName <- FromIr Text -> ReaderT EntityAlias FromIr Text
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (RelName -> FromIr Text
fromRelName RelName
_aarRelationshipName)
  Select
select <- do
    EntityAlias
lhsEntityAlias <- ReaderT EntityAlias FromIr EntityAlias
forall r (m :: * -> *). MonadReader r m => m r
ask
    FromIr Select -> ReaderT EntityAlias FromIr Select
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (Maybe (EntityAlias, HashMap ColumnName ColumnName)
-> AnnAggregateSelectG 'BigQuery Void Expression -> FromIr Select
fromSelectAggregate ((EntityAlias, HashMap ColumnName ColumnName)
-> Maybe (EntityAlias, HashMap ColumnName ColumnName)
forall a. a -> Maybe a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (EntityAlias
lhsEntityAlias, HashMap ColumnName ColumnName
mapping)) AnnAggregateSelectG 'BigQuery Void Expression
annSelectG)
  EntityAlias
alias <- FromIr EntityAlias -> ReaderT EntityAlias FromIr EntityAlias
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (NameTemplate -> FromIr EntityAlias
generateEntityAlias (Text -> NameTemplate
ArrayAggregateTemplate Text
joinFieldName))

  [(FieldName, FieldName)]
joinOn <- EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames EntityAlias
alias HashMap ColumnName ColumnName
mapping
  [(FieldName, FieldName)]
joinFields <- EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames (From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select)) HashMap ColumnName ColumnName
mapping
  NonEmpty Projection
joinFieldProjections <- [Projection] -> ReaderT EntityAlias FromIr (NonEmpty Projection)
forall (m :: * -> *) x.
MonadValidate (NonEmpty Error) m =>
[x] -> m (NonEmpty x)
toNonEmpty (((FieldName, FieldName) -> Projection)
-> [(FieldName, FieldName)] -> [Projection]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, FieldName) -> Projection
prepareJoinFieldProjection [(FieldName, FieldName)]
joinFields)

  let projections :: NonEmpty Projection
projections = Select -> NonEmpty Projection
selectProjections Select
select NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection
forall a. Semigroup a => a -> a -> a
<> NonEmpty Projection
joinFieldProjections
      joinSelect :: Select
joinSelect =
        Select
select
          { $sel:selectWhere:Select :: Where
selectWhere = Select -> Where
selectWhere Select
select,
            $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = ((FieldName, FieldName) -> FieldName)
-> [(FieldName, FieldName)] -> [FieldName]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, FieldName) -> FieldName
forall a b. (a, b) -> a
fst [(FieldName, FieldName)]
joinFields,
            $sel:selectProjections:Select :: NonEmpty Projection
selectProjections = NonEmpty Projection
projections
          }
  Join -> ReaderT EntityAlias FromIr Join
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    Join
      { $sel:joinAlias:Join :: EntityAlias
joinAlias = EntityAlias
alias,
        $sel:joinSource:Join :: JoinSource
joinSource = Select -> JoinSource
JoinSelect Select
joinSelect,
        $sel:joinRightTable:Join :: EntityAlias
joinRightTable = From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select),
        [(FieldName, FieldName)]
$sel:joinOn:Join :: [(FieldName, FieldName)]
joinOn :: [(FieldName, FieldName)]
joinOn,
        $sel:joinProvenance:Join :: JoinProvenance
joinProvenance =
          [(Text, FieldOrigin)] -> JoinProvenance
ArrayAggregateJoinProvenance
            ([(Text, FieldOrigin)] -> JoinProvenance)
-> [(Text, FieldOrigin)] -> JoinProvenance
forall a b. (a -> b) -> a -> b
$ (Projection -> Maybe (Text, FieldOrigin))
-> [Projection] -> [(Text, FieldOrigin)]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe (\Projection
p -> (,Projection -> FieldOrigin
aggregateProjectionsFieldOrigin Projection
p) (Text -> (Text, FieldOrigin))
-> Maybe Text -> Maybe (Text, FieldOrigin)
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Projection -> Maybe Text
projectionAlias Projection
p)
            ([Projection] -> [(Text, FieldOrigin)])
-> (Select -> [Projection]) -> Select -> [(Text, FieldOrigin)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. NonEmpty Projection -> [Projection]
forall a. NonEmpty a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList
            (NonEmpty Projection -> [Projection])
-> (Select -> NonEmpty Projection) -> Select -> [Projection]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. Select -> NonEmpty Projection
selectProjections
            (Select -> [(Text, FieldOrigin)])
-> Select -> [(Text, FieldOrigin)]
forall a b. (a -> b) -> a -> b
$ Select
select,
        -- Above: Needed by DataLoader to determine the type of
        -- Haskell-native join to perform.
        Text
$sel:joinFieldName:Join :: Text
joinFieldName :: Text
joinFieldName,
        $sel:joinExtractPath:Join :: Maybe Text
joinExtractPath = Maybe Text
forall a. Maybe a
Nothing,
        $sel:joinType:Join :: JoinType
joinType = case Nullable
nullable of Nullable
Nullable -> JoinType
LeftOuter; Nullable
NotNullable -> JoinType
Inner
      }
  where
    Ir.AnnRelationSelectG
      { RelName
$sel:_aarRelationshipName:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> RelName
_aarRelationshipName :: RelName
_aarRelationshipName,
        $sel:_aarColumnMapping:AnnRelationSelectG :: forall (b :: BackendType) a.
AnnRelationSelectG b a -> HashMap (Column b) (Column b)
_aarColumnMapping = HashMap ColumnName ColumnName
mapping :: HashMap ColumnName ColumnName,
        $sel:_aarAnnSelect:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> a
_aarAnnSelect = AnnAggregateSelectG 'BigQuery Void Expression
annSelectG,
        $sel:_aarNullable:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> Nullable
_aarNullable = Nullable
nullable
      } = ArrayAggregateSelectG 'BigQuery Void Expression
annRelationSelectG

-- | Produce a join for an array relation.
--
-- Array relations in PG/MSSQL are expressed using LEFT OUTER JOIN
-- LATERAL or OUTER APPLY, which are essentially producing for each
-- row on the left an array of the result from the right. Which is
-- absolutely what you want for the array relationship.
--
-- BigQuery doesn't support that. Therefore we are instead performing
-- one big array aggregation, for ALL rows in the table - there is no
-- join occurring on the left-hand-side table, grouped by join
-- fields. The data-loader will perform the LHS query and the RHS query
-- separately.
--
-- What we do have is a GROUP BY and make sure that the join fields
-- are included in the output. Finally, in the
-- DataLoader.Plan/DataLoader.Execute, we implement a Haskell-native
-- join of the left-hand-side table and the right-hand-side table.
--
-- Data looks like:
--
--     join_field_a | join_field_b | aggFieldName (array type)
--     1            | 1            | [ { x: 1, y: 2 }, ... ]
--     1            | 2            | [ { x: 1, y: 2 }, ... ]
--
-- etc.
--
-- We want to produce a query that looks like:
--
--     SELECT artist_other_id,  -- For joining.
--
--            array_agg(struct(album_self_id, title)) as aggFieldName
--
--            -- ^ Aggregating the actual data.
--
--     FROM (SELECT *,  -- Get everything, plus the row number:
--
--                  ROW_NUMBER() OVER(PARTITION BY artist_other_id) artist_album_index
--
--           FROM hasura.Album
--           ORDER BY album_self_id ASC
--
--           -- ^ Order by here is important for stable results.  Any
--           order by clauses for the album should appear here, NOT IN
--           THE ARRAY_AGG.
--
--           )
--
--           AS indexed_album
--
--     WHERE artist_album_index > 1
--     -- ^ Here is where offsetting occurs.
--
--     GROUP BY artist_other_id
--     -- ^ Group by for joining.
--
--     ORDER BY artist_other_id;
--     ^ Ordering for the artist table should appear here.
--
-- Note: if original select already uses a PARTITION BY internally (for distinct_on)
-- join fields are added to partition expressions to give proper semantics of distinct_on
-- combined with an array relation
fromArrayRelationSelectG ::
  Ir.ArrayRelationSelectG 'BigQuery Void Expression ->
  ReaderT EntityAlias FromIr Join
fromArrayRelationSelectG :: ArrayRelationSelectG 'BigQuery Void Expression
-> ReaderT EntityAlias FromIr Join
fromArrayRelationSelectG ArrayRelationSelectG 'BigQuery Void Expression
annRelationSelectG = do
  PartitionableSelect
pselect <- (FromIr PartitionableSelect
-> ReaderT EntityAlias FromIr PartitionableSelect
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (FromIr PartitionableSelect
 -> ReaderT EntityAlias FromIr PartitionableSelect)
-> (EntityAlias -> FromIr PartitionableSelect)
-> EntityAlias
-> ReaderT EntityAlias FromIr PartitionableSelect
forall b c a. (b -> c) -> (a -> b) -> a -> c
. (ParentSelectFromEntity
 -> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
 -> FromIr PartitionableSelect)
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> ParentSelectFromEntity
-> FromIr PartitionableSelect
forall a b c. (a -> b -> c) -> b -> a -> c
flip ParentSelectFromEntity
-> AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
-> FromIr PartitionableSelect
fromSelectRows AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSelectG (ParentSelectFromEntity -> FromIr PartitionableSelect)
-> (EntityAlias -> ParentSelectFromEntity)
-> EntityAlias
-> FromIr PartitionableSelect
forall b c a. (b -> c) -> (a -> b) -> a -> c
. EntityAlias -> ParentSelectFromEntity
ParentEntityAlias) (EntityAlias -> ReaderT EntityAlias FromIr PartitionableSelect)
-> ReaderT EntityAlias FromIr EntityAlias
-> ReaderT EntityAlias FromIr PartitionableSelect
forall (m :: * -> *) a b. Monad m => (a -> m b) -> m a -> m b
=<< ReaderT EntityAlias FromIr EntityAlias
forall r (m :: * -> *). MonadReader r m => m r
ask -- Take the original select.
  Text
joinFieldName <- FromIr Text -> ReaderT EntityAlias FromIr Text
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (RelName -> FromIr Text
fromRelName RelName
_aarRelationshipName)
  EntityAlias
alias <- FromIr EntityAlias -> ReaderT EntityAlias FromIr EntityAlias
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (NameTemplate -> FromIr EntityAlias
generateEntityAlias (Text -> NameTemplate
ArrayRelationTemplate Text
joinFieldName))
  EntityAlias
indexAlias <- FromIr EntityAlias -> ReaderT EntityAlias FromIr EntityAlias
forall (m :: * -> *) a. Monad m => m a -> ReaderT EntityAlias m a
forall (t :: (* -> *) -> * -> *) (m :: * -> *) a.
(MonadTrans t, Monad m) =>
m a -> t m a
lift (NameTemplate -> FromIr EntityAlias
generateEntityAlias NameTemplate
IndexTemplate)
  [(FieldName, FieldName)]
joinOn <- EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames EntityAlias
alias HashMap ColumnName ColumnName
mapping
  NonEmpty (FieldName, FieldName)
joinFields <- EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames (From -> EntityAlias
fromAlias (PartitionableSelect -> From
pselectFrom PartitionableSelect
pselect)) HashMap ColumnName ColumnName
mapping ReaderT EntityAlias FromIr [(FieldName, FieldName)]
-> ([(FieldName, FieldName)]
    -> ReaderT EntityAlias FromIr (NonEmpty (FieldName, FieldName)))
-> ReaderT EntityAlias FromIr (NonEmpty (FieldName, FieldName))
forall a b.
ReaderT EntityAlias FromIr a
-> (a -> ReaderT EntityAlias FromIr b)
-> ReaderT EntityAlias FromIr b
forall (m :: * -> *) a b. Monad m => m a -> (a -> m b) -> m b
>>= [(FieldName, FieldName)]
-> ReaderT EntityAlias FromIr (NonEmpty (FieldName, FieldName))
forall (m :: * -> *) x.
MonadValidate (NonEmpty Error) m =>
[x] -> m (NonEmpty x)
toNonEmpty
  let select :: Select
select = PartitionableSelect -> [FieldName] -> Select
withExtraPartitionFields PartitionableSelect
pselect ([FieldName] -> Select) -> [FieldName] -> Select
forall a b. (a -> b) -> a -> b
$ NonEmpty FieldName -> [FieldName]
forall a. NonEmpty a -> [a]
NE.toList (((FieldName, FieldName) -> FieldName)
-> NonEmpty (FieldName, FieldName) -> NonEmpty FieldName
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (FieldName, FieldName) -> FieldName
forall a b. (a, b) -> a
fst NonEmpty (FieldName, FieldName)
joinFields)
      joinFieldProjections :: NonEmpty Projection
joinFieldProjections = ((FieldName, FieldName) -> Projection)
-> NonEmpty (FieldName, FieldName) -> NonEmpty Projection
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (FieldName, FieldName) -> Projection
prepareJoinFieldProjection NonEmpty (FieldName, FieldName)
joinFields

  let joinSelect :: Select
joinSelect =
        Select
          { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
            $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
One,
            $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
            $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields = Select -> Maybe [Text]
selectFinalWantedFields Select
select,
            $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
            $sel:selectProjections:Select :: NonEmpty Projection
selectProjections =
              NonEmpty Projection
joinFieldProjections
                NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection
forall a. Semigroup a => a -> a -> a
<> Projection -> NonEmpty Projection
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
                  ( Aliased ArrayAgg -> Projection
ArrayAggProjection
                      Aliased
                        { $sel:aliasedThing:Aliased :: ArrayAgg
aliasedThing =
                            ArrayAgg
                              { $sel:arrayAggProjections:ArrayAgg :: NonEmpty Projection
arrayAggProjections =
                                  (Projection -> Projection)
-> NonEmpty Projection -> NonEmpty Projection
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
                                    (EntityAlias -> Projection -> Projection
aliasToFieldProjection (From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select)))
                                    (Select -> NonEmpty Projection
selectProjections Select
select),
                                $sel:arrayAggOrderBy:ArrayAgg :: Maybe (NonEmpty OrderBy)
arrayAggOrderBy = Select -> Maybe (NonEmpty OrderBy)
selectOrderBy Select
select,
                                $sel:arrayAggTop:ArrayAgg :: Top
arrayAggTop = Select -> Top
selectTop Select
select
                                -- The sub-select takes care of caring about global top.
                                --
                                -- This handles the LIMIT need.
                              },
                          $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
aggFieldName
                        }
                  ),
            $sel:selectFrom:Select :: From
selectFrom =
              Aliased Select -> From
FromSelect
                ( Aliased
                    { $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
forall a b. Coercible a b => a -> b
coerce (From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select)),
                      $sel:aliasedThing:Aliased :: Select
aliasedThing =
                        Select
                          { $sel:selectWith:Select :: Maybe With
selectWith = Maybe With
forall a. Maybe a
Nothing,
                            $sel:selectProjections:Select :: NonEmpty Projection
selectProjections =
                              Select -> NonEmpty Projection
selectProjections Select
select
                                NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection
forall a. Semigroup a => a -> a -> a
<> NonEmpty Projection
joinFieldProjections
                                NonEmpty Projection -> [Projection] -> NonEmpty Projection
forall a. NonEmpty a -> [a] -> NonEmpty a
`appendToNonEmpty` forall (t :: * -> *) m a.
(Foldable t, Monoid m) =>
(a -> m) -> t a -> m
foldMap @Maybe
                                  ( (OrderBy -> Projection) -> [OrderBy] -> [Projection]
forall a b. (a -> b) -> [a] -> [b]
map \OrderBy {FieldName
$sel:orderByFieldName:OrderBy :: OrderBy -> FieldName
orderByFieldName :: FieldName
orderByFieldName} ->
                                      Aliased FieldName -> Projection
FieldNameProjection
                                        Aliased
                                          { $sel:aliasedThing:Aliased :: FieldName
aliasedThing = FieldName
orderByFieldName,
                                            $sel:aliasedAlias:Aliased :: Text
aliasedAlias = FieldName -> Text
fieldName FieldName
orderByFieldName
                                          }
                                  )
                                  (NonEmpty OrderBy -> [OrderBy]
forall a. NonEmpty a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList (NonEmpty OrderBy -> [OrderBy])
-> Maybe (NonEmpty OrderBy) -> Maybe [OrderBy]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
<$> Select -> Maybe (NonEmpty OrderBy)
selectOrderBy Select
select)
                                -- Above: Select "order by" fields as they're being used
                                -- inside `ARRAY_AGG` function (as ORDER BY clause)
                                NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection
forall a. Semigroup a => a -> a -> a
<> Projection -> NonEmpty Projection
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
                                  ( Aliased WindowFunction -> Projection
WindowProjection
                                      ( Aliased
                                          { $sel:aliasedAlias:Aliased :: Text
aliasedAlias = EntityAlias -> Text
unEntityAlias EntityAlias
indexAlias,
                                            $sel:aliasedThing:Aliased :: WindowFunction
aliasedThing =
                                              NonEmpty FieldName -> Maybe (NonEmpty OrderBy) -> WindowFunction
RowNumberOverPartitionBy
                                                -- The row numbers start from 1.
                                                ( ((FieldName, FieldName) -> FieldName)
-> NonEmpty (FieldName, FieldName) -> NonEmpty FieldName
forall a b. (a -> b) -> NonEmpty a -> NonEmpty b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (FieldName, FieldName) -> FieldName
forall a b. (a, b) -> a
fst NonEmpty (FieldName, FieldName)
joinFields
                                                )
                                                (Select -> Maybe (NonEmpty OrderBy)
selectOrderBy Select
select)
                                                -- Above: Having the order by
                                                -- in here ensures that the
                                                -- row numbers are ordered by
                                                -- this ordering. Below, we
                                                -- order again for the
                                                -- general row order. Both
                                                -- are needed!
                                          }
                                      )
                                  ),
                            $sel:selectFrom:Select :: From
selectFrom = Select -> From
selectFrom Select
select,
                            $sel:selectJoins:Select :: [Join]
selectJoins = Select -> [Join]
selectJoins Select
select,
                            $sel:selectWhere:Select :: Where
selectWhere = Select -> Where
selectWhere Select
select,
                            $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Select -> Maybe (NonEmpty OrderBy)
selectOrderBy Select
select,
                            -- Above: This orders the rows themselves. In
                            -- the RowNumberOverPartitionBy, we also set
                            -- a row order for the calculation of the
                            -- indices. Both are needed!
                            $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing,
                            $sel:selectFinalWantedFields:Select :: Maybe [Text]
selectFinalWantedFields =
                              Select -> Maybe [Text]
selectFinalWantedFields Select
select,
                            $sel:selectCardinality:Select :: Cardinality
selectCardinality = Cardinality
Many,
                            $sel:selectAsStruct:Select :: AsStruct
selectAsStruct = AsStruct
NoAsStruct,
                            $sel:selectTop:Select :: Top
selectTop = Top
NoTop,
                            $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = [FieldName]
forall a. Monoid a => a
mempty
                          }
                    }
                ),
            $sel:selectWhere:Select :: Where
selectWhere =
              case Select -> Maybe Expression
selectOffset Select
select of
                Maybe Expression
Nothing -> Where
forall a. Monoid a => a
mempty
                Just Expression
offset ->
                  [Expression] -> Where
Where
                    [ Op -> Expression -> Expression -> Expression
OpExpression
                        Op
MoreOp
                        (FieldName -> Expression
ColumnExpression FieldName {$sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = EntityAlias -> Text
forall a b. Coercible a b => a -> b
coerce (From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select)), $sel:fieldName:FieldName :: Text
fieldName = EntityAlias -> Text
unEntityAlias EntityAlias
indexAlias})
                        Expression
offset
                    ],
            $sel:selectOrderBy:Select :: Maybe (NonEmpty OrderBy)
selectOrderBy = Maybe (NonEmpty OrderBy)
forall a. Maybe a
Nothing, -- Not needed.
            $sel:selectJoins:Select :: [Join]
selectJoins = [Join]
forall a. Monoid a => a
mempty,
            $sel:selectOffset:Select :: Maybe Expression
selectOffset = Maybe Expression
forall a. Maybe a
Nothing,
            -- This group by corresponds to the field name projections above. E.g. artist_other_id
            $sel:selectGroupBy:Select :: [FieldName]
selectGroupBy = ((FieldName, FieldName) -> FieldName)
-> [(FieldName, FieldName)] -> [FieldName]
forall a b. (a -> b) -> [a] -> [b]
map (FieldName, FieldName) -> FieldName
forall a b. (a, b) -> a
fst (NonEmpty (FieldName, FieldName) -> [(FieldName, FieldName)]
forall a. NonEmpty a -> [a]
NE.toList NonEmpty (FieldName, FieldName)
joinFields)
          }
  Join -> ReaderT EntityAlias FromIr Join
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
    Join
      { $sel:joinAlias:Join :: EntityAlias
joinAlias = EntityAlias
alias,
        $sel:joinSource:Join :: JoinSource
joinSource = Select -> JoinSource
JoinSelect Select
joinSelect,
        $sel:joinRightTable:Join :: EntityAlias
joinRightTable = From -> EntityAlias
fromAlias (Select -> From
selectFrom Select
select),
        [(FieldName, FieldName)]
$sel:joinOn:Join :: [(FieldName, FieldName)]
joinOn :: [(FieldName, FieldName)]
joinOn,
        $sel:joinProvenance:Join :: JoinProvenance
joinProvenance =
          [Text] -> JoinProvenance
ArrayJoinProvenance
            ( if Bool
True
                then ([Text] -> Maybe [Text] -> [Text]
forall a. a -> Maybe a -> a
fromMaybe [] (Select -> Maybe [Text]
selectFinalWantedFields Select
select))
                else
                  ( (Projection -> Maybe Text) -> [Projection] -> [Text]
forall a b. (a -> Maybe b) -> [a] -> [b]
forall (f :: * -> *) a b.
Filterable f =>
(a -> Maybe b) -> f a -> f b
mapMaybe
                      Projection -> Maybe Text
projectionAlias
                      (NonEmpty Projection -> [Projection]
forall a. NonEmpty a -> [a]
forall (t :: * -> *) a. Foldable t => t a -> [a]
toList (Select -> NonEmpty Projection
selectProjections Select
select))
                  )
            ),
        -- Above: Needed by DataLoader to determine the type of
        -- Haskell-native join to perform.
        Text
$sel:joinFieldName:Join :: Text
joinFieldName :: Text
joinFieldName,
        $sel:joinExtractPath:Join :: Maybe Text
joinExtractPath = Text -> Maybe Text
forall a. a -> Maybe a
Just Text
aggFieldName,
        $sel:joinType:Join :: JoinType
joinType = case Nullable
nullable of Nullable
Nullable -> JoinType
LeftOuter; Nullable
NotNullable -> JoinType
Inner
      }
  where
    Ir.AnnRelationSelectG
      { RelName
$sel:_aarRelationshipName:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> RelName
_aarRelationshipName :: RelName
_aarRelationshipName,
        $sel:_aarColumnMapping:AnnRelationSelectG :: forall (b :: BackendType) a.
AnnRelationSelectG b a -> HashMap (Column b) (Column b)
_aarColumnMapping = HashMap ColumnName ColumnName
mapping :: HashMap ColumnName ColumnName,
        $sel:_aarAnnSelect:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> a
_aarAnnSelect = AnnSelectG 'BigQuery (AnnFieldG 'BigQuery Void) Expression
annSelectG,
        $sel:_aarNullable:AnnRelationSelectG :: forall (b :: BackendType) a. AnnRelationSelectG b a -> Nullable
_aarNullable = Nullable
nullable
      } = ArrayRelationSelectG 'BigQuery Void Expression
annRelationSelectG

-- | For entity projections, convert any entity aliases to their field
-- names. ArrayEntityProjection and ExpressionProjection get converted
-- to aliases to fields with the same names as all the expressions
-- have already aliases applied in select from ArrayAgg
-- (created in Hasura.Backends.BigQuery.ToQuery.fromArrayAgg)
aliasToFieldProjection :: EntityAlias -> Projection -> Projection
aliasToFieldProjection :: EntityAlias -> Projection -> Projection
aliasToFieldProjection (EntityAlias Text
selectAlias) =
  \case
    EntityProjection Aliased {$sel:aliasedAlias:Aliased :: forall a. Aliased a -> Text
aliasedAlias = Text
name, $sel:aliasedThing:Aliased :: forall a. Aliased a -> a
aliasedThing = [(FieldName, FieldOrigin)]
fields} ->
      Aliased [(FieldName, FieldOrigin)] -> Projection
EntityProjection
        Aliased
          { $sel:aliasedAlias:Aliased :: Text
aliasedAlias = Text
name,
            $sel:aliasedThing:Aliased :: [(FieldName, FieldOrigin)]
aliasedThing =
              ((FieldName, FieldOrigin) -> (FieldName, FieldOrigin))
-> [(FieldName, FieldOrigin)] -> [(FieldName, FieldOrigin)]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap
                (\(FieldName {Text
$sel:fieldNameEntity:FieldName :: FieldName -> Text
$sel:fieldName:FieldName :: FieldName -> Text
fieldName :: Text
fieldNameEntity :: Text
..}, FieldOrigin
origin) -> (FieldName {$sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = Text
name, Text
$sel:fieldName:FieldName :: Text
fieldName :: Text
..}, FieldOrigin
origin))
                [(FieldName, FieldOrigin)]
fields
          }
    ArrayEntityProjection EntityAlias
_ Aliased [FieldName]
aliased ->
      Aliased [FieldName] -> Projection
forall a. Aliased a -> Projection
aliasColumn Aliased [FieldName]
aliased
    ExpressionProjection Aliased Expression
aliased ->
      Aliased Expression -> Projection
forall a. Aliased a -> Projection
aliasColumn Aliased Expression
aliased
    Projection
p -> Projection
p
  where
    aliasColumn :: Aliased a -> Projection
    aliasColumn :: forall a. Aliased a -> Projection
aliasColumn Aliased a
aliased =
      Aliased Expression -> Projection
ExpressionProjection
        Aliased a
aliased
          { $sel:aliasedThing:Aliased :: Expression
aliasedThing = FieldName -> Expression
ColumnExpression (FieldName {$sel:fieldName:FieldName :: Text
fieldName = Aliased a -> Text
forall a. Aliased a -> Text
aliasedAlias Aliased a
aliased, $sel:fieldNameEntity:FieldName :: Text
fieldNameEntity = Text
selectAlias})
          }

fromRelName :: Rql.RelName -> FromIr Text
fromRelName :: RelName -> FromIr Text
fromRelName RelName
relName =
  Text -> FromIr Text
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (RelName -> Text
Rql.relNameToTxt RelName
relName)

-- | The context given by the reader is of the previous/parent
-- "remote" table. The WHERE that we're generating goes in the child,
-- "local" query. The @From@ passed in as argument is the local table.
--
-- We should hope to see e.g. "post.category = category.id" for a
-- local table of post and a remote table of category.
--
-- The left/right columns in @HashMap ColumnName ColumnName@ corresponds
-- to the left/right of @select ... join ...@. Therefore left=remote,
-- right=local in this context.
fromMapping ::
  From ->
  HashMap ColumnName ColumnName ->
  ReaderT EntityAlias FromIr [Expression]
fromMapping :: From
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [Expression]
fromMapping From
localFrom =
  ((ColumnName, ColumnName) -> ReaderT EntityAlias FromIr Expression)
-> [(ColumnName, ColumnName)]
-> ReaderT EntityAlias FromIr [Expression]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse
    ( \(ColumnName
remoteColumn, ColumnName
localColumn) -> do
        FieldName
localFieldName <- (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr FieldName
-> ReaderT EntityAlias FromIr FieldName
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const (From -> EntityAlias
fromAlias From
localFrom)) (ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn ColumnName
localColumn)
        FieldName
remoteFieldName <- ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn ColumnName
remoteColumn
        Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure
          ( Expression -> Expression -> Expression
EqualExpression
              (FieldName -> Expression
ColumnExpression FieldName
localFieldName)
              (FieldName -> Expression
ColumnExpression FieldName
remoteFieldName)
          )
    )
    ([(ColumnName, ColumnName)]
 -> ReaderT EntityAlias FromIr [Expression])
-> (HashMap ColumnName ColumnName -> [(ColumnName, ColumnName)])
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [Expression]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. HashMap ColumnName ColumnName -> [(ColumnName, ColumnName)]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList

-- | Given an alias for the remote table, and a map of local-to-remote column
-- name pairings, produce 'FieldName' pairings (column names paired with their
-- associated table names).
--
-- For example, we might convert the following:
--
-- @
-- [ ( ColumnName { columnName = "author_id" }
--   , ColumnName { columnName = "id" }
--   )
-- ]
-- @
--
-- ... into something like this:
--
-- @
-- ( FieldName
--     { fieldName = "id"
--     , fieldNameEntity = "t_author1"
--     }
-- , FieldName
--     { fieldName = "author_id"
--     , fieldNameEntity = "t_article1"
--     }
-- )
-- @
--
-- Note that the columns __flip around__ for the output. The input map is
-- @(local, remote)@.
fromMappingFieldNames ::
  EntityAlias ->
  HashMap ColumnName ColumnName ->
  ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames :: EntityAlias
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
fromMappingFieldNames EntityAlias
remoteFrom = ((ColumnName, ColumnName)
 -> ReaderT EntityAlias FromIr (FieldName, FieldName))
-> [(ColumnName, ColumnName)]
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse (ColumnName, ColumnName)
-> ReaderT EntityAlias FromIr (FieldName, FieldName)
go ([(ColumnName, ColumnName)]
 -> ReaderT EntityAlias FromIr [(FieldName, FieldName)])
-> (HashMap ColumnName ColumnName -> [(ColumnName, ColumnName)])
-> HashMap ColumnName ColumnName
-> ReaderT EntityAlias FromIr [(FieldName, FieldName)]
forall b c a. (b -> c) -> (a -> b) -> a -> c
. HashMap ColumnName ColumnName -> [(ColumnName, ColumnName)]
forall k v. HashMap k v -> [(k, v)]
HashMap.toList
  where
    go :: (ColumnName, ColumnName)
-> ReaderT EntityAlias FromIr (FieldName, FieldName)
go (ColumnName
localColumn, ColumnName
remoteColumn) = do
      FieldName
remoteFieldName <- (EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr FieldName
-> ReaderT EntityAlias FromIr FieldName
forall a.
(EntityAlias -> EntityAlias)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr a
forall r (m :: * -> *) a. MonadReader r m => (r -> r) -> m a -> m a
local (EntityAlias -> EntityAlias -> EntityAlias
forall a b. a -> b -> a
const EntityAlias
remoteFrom) (ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn ColumnName
remoteColumn)
      FieldName
localFieldName <- ColumnName -> ReaderT EntityAlias FromIr FieldName
fromColumn ColumnName
localColumn
      (FieldName, FieldName)
-> ReaderT EntityAlias FromIr (FieldName, FieldName)
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (FieldName
remoteFieldName, FieldName
localFieldName)

--------------------------------------------------------------------------------
-- Basic SQL expression types

fromOpExpG :: Expression -> Ir.OpExpG 'BigQuery Expression -> FromIr Expression
fromOpExpG :: Expression -> OpExpG 'BigQuery Expression -> FromIr Expression
fromOpExpG Expression
expression OpExpG 'BigQuery Expression
op =
  case OpExpG 'BigQuery Expression
op of
    OpExpG 'BigQuery Expression
Ir.ANISNULL -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression
IsNullExpression Expression
expression)
    OpExpG 'BigQuery Expression
Ir.ANISNOTNULL -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression
IsNotNullExpression Expression
expression)
    Ir.AEQ ComparisonNullability
Ir.NullableComparison Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression -> Expression
nullableBoolEquality Expression
expression Expression
val)
    Ir.AEQ ComparisonNullability
Ir.NonNullableComparison Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression -> Expression
EqualExpression Expression
expression Expression
val)
    Ir.ANE ComparisonNullability
Ir.NullableComparison Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression -> Expression
nullableBoolInequality Expression
expression Expression
val)
    Ir.ANE ComparisonNullability
Ir.NonNullableComparison Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> Expression -> Expression
NotEqualExpression Expression
expression Expression
val)
    Ir.AIN Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
InOp Expression
expression Expression
val)
    Ir.ANIN Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
NotInOp Expression
expression Expression
val)
    Ir.AGT Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
MoreOp Expression
expression Expression
val)
    Ir.ALT Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
LessOp Expression
expression Expression
val)
    Ir.AGTE Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
MoreOrEqualOp Expression
expression Expression
val)
    Ir.ALTE Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
LessOrEqualOp Expression
expression Expression
val)
    Ir.ACast CastExp 'BigQuery Expression
_casts -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))
    Ir.ALIKE Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
LikeOp Expression
expression Expression
val)
    Ir.ANLIKE Expression
val -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Op -> Expression -> Expression -> Expression
OpExpression Op
NotLikeOp Expression
expression Expression
val)
    Ir.ABackendSpecific BooleanOperators 'BigQuery Expression
op' -> Expression -> FromIr Expression
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Expression -> BooleanOperators Expression -> Expression
fromBackendSpecificOpExpG Expression
expression BooleanOperators 'BigQuery Expression
BooleanOperators Expression
op')
    Ir.CEQ RootOrCurrentColumn 'BigQuery
_rhsCol -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))
    Ir.CNE RootOrCurrentColumn 'BigQuery
_rhsCol -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))
    Ir.CGT RootOrCurrentColumn 'BigQuery
_rhsCol -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))
    Ir.CLT RootOrCurrentColumn 'BigQuery
_rhsCol -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))
    Ir.CGTE RootOrCurrentColumn 'BigQuery
_rhsCol -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))
    Ir.CLTE RootOrCurrentColumn 'BigQuery
_rhsCol -> NonEmpty Error -> FromIr Expression
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (OpExpG 'BigQuery Expression -> Error
UnsupportedOpExpG OpExpG 'BigQuery Expression
op))

fromBackendSpecificOpExpG :: Expression -> BigQuery.BooleanOperators Expression -> Expression
fromBackendSpecificOpExpG :: Expression -> BooleanOperators Expression -> Expression
fromBackendSpecificOpExpG Expression
expression BooleanOperators Expression
op =
  let func :: Text -> Expression -> Expression
func Text
name Expression
val = FunctionName -> [Expression] -> Expression
FunctionExpression (Text -> Maybe Text -> FunctionName
FunctionName Text
name Maybe Text
forall a. Maybe a
Nothing) [Expression
expression, Expression
val]
   in case BooleanOperators Expression
op of
        BigQuery.ASTContains Expression
v -> Text -> Expression -> Expression
func Text
"ST_CONTAINS" Expression
v
        BigQuery.ASTEquals Expression
v -> Text -> Expression -> Expression
func Text
"ST_EQUALS" Expression
v
        BigQuery.ASTTouches Expression
v -> Text -> Expression -> Expression
func Text
"ST_TOUCHES" Expression
v
        BigQuery.ASTWithin Expression
v -> Text -> Expression -> Expression
func Text
"ST_WITHIN" Expression
v
        BigQuery.ASTIntersects Expression
v -> Text -> Expression -> Expression
func Text
"ST_INTERSECTS" Expression
v
        BigQuery.ASTDWithin (Ir.DWithinGeogOp Expression
r Expression
v Expression
sph) ->
          FunctionName -> [Expression] -> Expression
FunctionExpression (Text -> Maybe Text -> FunctionName
FunctionName Text
"ST_DWITHIN" Maybe Text
forall a. Maybe a
Nothing) [Expression
expression, Expression
v, Expression
r, Expression
sph]

nullableBoolEquality :: Expression -> Expression -> Expression
nullableBoolEquality :: Expression -> Expression -> Expression
nullableBoolEquality Expression
x Expression
y =
  [Expression] -> Expression
OrExpression
    [ Expression -> Expression -> Expression
EqualExpression Expression
x Expression
y,
      [Expression] -> Expression
AndExpression [Expression -> Expression
IsNullExpression Expression
x, Expression -> Expression
IsNullExpression Expression
y]
    ]

nullableBoolInequality :: Expression -> Expression -> Expression
nullableBoolInequality :: Expression -> Expression -> Expression
nullableBoolInequality Expression
x Expression
y =
  [Expression] -> Expression
OrExpression
    [ Expression -> Expression -> Expression
NotEqualExpression Expression
x Expression
y,
      [Expression] -> Expression
AndExpression [Expression -> Expression
IsNotNullExpression Expression
x, Expression -> Expression
IsNullExpression Expression
y]
    ]

fromGBoolExp :: Ir.GBoolExp 'BigQuery Expression -> ReaderT EntityAlias FromIr Expression
fromGBoolExp :: GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp =
  \case
    Ir.BoolAnd [GBoolExp 'BigQuery Expression]
expressions ->
      ([Expression] -> Expression)
-> ReaderT EntityAlias FromIr [Expression]
-> ReaderT EntityAlias FromIr Expression
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap [Expression] -> Expression
AndExpression ((GBoolExp 'BigQuery Expression
 -> ReaderT EntityAlias FromIr Expression)
-> [GBoolExp 'BigQuery Expression]
-> ReaderT EntityAlias FromIr [Expression]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp [GBoolExp 'BigQuery Expression]
expressions)
    Ir.BoolOr [GBoolExp 'BigQuery Expression]
expressions ->
      ([Expression] -> Expression)
-> ReaderT EntityAlias FromIr [Expression]
-> ReaderT EntityAlias FromIr Expression
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap [Expression] -> Expression
OrExpression ((GBoolExp 'BigQuery Expression
 -> ReaderT EntityAlias FromIr Expression)
-> [GBoolExp 'BigQuery Expression]
-> ReaderT EntityAlias FromIr [Expression]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp [GBoolExp 'BigQuery Expression]
expressions)
    Ir.BoolNot GBoolExp 'BigQuery Expression
expression -> (Expression -> Expression)
-> ReaderT EntityAlias FromIr Expression
-> ReaderT EntityAlias FromIr Expression
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Expression -> Expression
NotExpression (GBoolExp 'BigQuery Expression
-> ReaderT EntityAlias FromIr Expression
fromGBoolExp GBoolExp 'BigQuery Expression
expression)
    Ir.BoolExists GExists 'BigQuery Expression
gExists -> (Select -> Expression)
-> ReaderT EntityAlias FromIr Select
-> ReaderT EntityAlias FromIr Expression
forall a b.
(a -> b)
-> ReaderT EntityAlias FromIr a -> ReaderT EntityAlias FromIr b
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap Select -> Expression
ExistsExpression (GExists 'BigQuery Expression -> ReaderT EntityAlias FromIr Select
fromGExists GExists 'BigQuery Expression
gExists)
    Ir.BoolField Expression
expression -> Expression -> ReaderT EntityAlias FromIr Expression
forall a. a -> ReaderT EntityAlias FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Expression
expression

--------------------------------------------------------------------------------
-- Misc combinators

-- | Attempt to refine a list into a 'NonEmpty'. If the given list is empty,
-- this will 'refute' the computation with an 'UnexpectedEmptyList' error.
toNonEmpty :: (MonadValidate (NonEmpty Error) m) => [x] -> m (NonEmpty x)
toNonEmpty :: forall (m :: * -> *) x.
MonadValidate (NonEmpty Error) m =>
[x] -> m (NonEmpty x)
toNonEmpty = \case
  [] -> NonEmpty Error -> m (NonEmpty x)
forall a. NonEmpty Error -> m a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error
UnexpectedEmptyList Error -> [Error] -> NonEmpty Error
forall a. a -> [a] -> NonEmpty a
:| [])
  x
x : [x]
xs -> NonEmpty x -> m (NonEmpty x)
forall a. a -> m a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (x
x x -> [x] -> NonEmpty x
forall a. a -> [a] -> NonEmpty a
:| [x]
xs)

-- | Get the remote field from a pair (see 'fromMappingFieldNames' for more
-- information) and produce a 'Projection'.
prepareJoinFieldProjection :: (FieldName, FieldName) -> Projection
prepareJoinFieldProjection :: (FieldName, FieldName) -> Projection
prepareJoinFieldProjection (FieldName
fieldName', FieldName
_) =
  Aliased FieldName -> Projection
FieldNameProjection
    Aliased
      { $sel:aliasedThing:Aliased :: FieldName
aliasedThing = FieldName
fieldName',
        $sel:aliasedAlias:Aliased :: Text
aliasedAlias = FieldName -> Text
fieldName FieldName
fieldName'
      }

selectProjectionsFromFieldSources :: Bool -> [FieldSource] -> FromIr (NonEmpty Projection)
selectProjectionsFromFieldSources :: Bool -> [FieldSource] -> FromIr (NonEmpty Projection)
selectProjectionsFromFieldSources Bool
keepJoinField [FieldSource]
fieldSources = do
  Maybe (NonEmpty (NonEmpty Projection))
projections <- FromIr (NonEmpty (NonEmpty Projection))
-> FromIr (Maybe (NonEmpty (NonEmpty Projection)))
forall a. FromIr a -> FromIr (Maybe a)
forall e (m :: * -> *) a. MonadValidate e m => m a -> m (Maybe a)
tolerate do
    [NonEmpty Projection]
projections' <- (FieldSource -> FromIr (NonEmpty Projection))
-> [FieldSource] -> FromIr [NonEmpty Projection]
forall (t :: * -> *) (f :: * -> *) a b.
(Traversable t, Applicative f) =>
(a -> f b) -> t a -> f (t b)
forall (f :: * -> *) a b.
Applicative f =>
(a -> f b) -> [a] -> f [b]
traverse (Bool -> FieldSource -> FromIr (NonEmpty Projection)
fieldSourceProjections Bool
keepJoinField) [FieldSource]
fieldSources
    [NonEmpty Projection] -> FromIr (NonEmpty (NonEmpty Projection))
forall (m :: * -> *) x.
MonadValidate (NonEmpty Error) m =>
[x] -> m (NonEmpty x)
toNonEmpty [NonEmpty Projection]
projections'

  case Maybe (NonEmpty (NonEmpty Projection))
projections of
    Just (NonEmpty Projection
x :| [NonEmpty Projection]
xs) -> NonEmpty Projection -> FromIr (NonEmpty Projection)
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure ((NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection)
-> NonEmpty Projection
-> [NonEmpty Projection]
-> NonEmpty Projection
forall b a. (b -> a -> b) -> b -> [a] -> b
forall (t :: * -> *) b a.
Foldable t =>
(b -> a -> b) -> b -> t a -> b
foldl' NonEmpty Projection -> NonEmpty Projection -> NonEmpty Projection
forall a. Semigroup a => a -> a -> a
(<>) NonEmpty Projection
x [NonEmpty Projection]
xs)
    Maybe (NonEmpty (NonEmpty Projection))
Nothing -> NonEmpty Error -> FromIr (NonEmpty Projection)
forall a. NonEmpty Error -> FromIr a
forall e (m :: * -> *) a. MonadValidate e m => e -> m a
refute (Error -> NonEmpty Error
forall a. a -> NonEmpty a
forall (f :: * -> *) a. Applicative f => a -> f a
pure Error
NoProjectionFields)

trueExpression :: Expression
trueExpression :: Expression
trueExpression = TypedValue -> Expression
ValueExpression (ScalarType -> Value -> TypedValue
TypedValue ScalarType
BoolScalarType (Bool -> Value
BoolValue Bool
True))

--------------------------------------------------------------------------------
-- Constants

aggFieldName :: Text
aggFieldName :: Text
aggFieldName = Text
"agg"

existsFieldName :: Text
existsFieldName :: Text
existsFieldName = Text
"exists_placeholder"

--------------------------------------------------------------------------------
-- Name generation

data NameTemplate
  = ArrayRelationTemplate Text
  | ArrayAggregateTemplate Text
  | ObjectRelationTemplate Text
  | TableTemplate Text
  | ForOrderAlias Text
  | IndexTemplate
  | UnnestTemplate
  | FunctionTemplate FunctionName
  | NativeQueryTemplate NativeQueryName

generateEntityAlias :: NameTemplate -> FromIr EntityAlias
generateEntityAlias :: NameTemplate -> FromIr EntityAlias
generateEntityAlias NameTemplate
template = do
  ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  ()
-> FromIr ()
forall a.
ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  a
-> FromIr a
FromIr
    ( (FromIrState -> FromIrState)
-> ReaderT
     FromIrReader
     (StateT
        FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
     ()
forall s (m :: * -> *). MonadState s m => (s -> s) -> m ()
modify'
        ( \FromIrState {Map Text Int
indices :: FromIrState -> Map Text Int
indices :: Map Text Int
..} ->
            FromIrState {indices :: Map Text Int
indices = (Int -> Int -> Int) -> Text -> Int -> Map Text Int -> Map Text Int
forall k a. Ord k => (a -> a -> a) -> k -> a -> Map k a -> Map k a
M.insertWith Int -> Int -> Int
forall a. Num a => a -> a -> a
(+) Text
prefix Int
start Map Text Int
indices, ..}
        )
    )
  Map Text Int
i <- ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  (Map Text Int)
-> FromIr (Map Text Int)
forall a.
ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  a
-> FromIr a
FromIr ((FromIrState -> Map Text Int)
-> ReaderT
     FromIrReader
     (StateT
        FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
     (Map Text Int)
forall s (m :: * -> *) a. MonadState s m => (s -> a) -> m a
gets FromIrState -> Map Text Int
indices)
  EntityAlias -> FromIr EntityAlias
forall a. a -> FromIr a
forall (f :: * -> *) a. Applicative f => a -> f a
pure (Text -> EntityAlias
EntityAlias (Text
prefix Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Int -> Text
forall a. Show a => a -> Text
tshow (Int -> Maybe Int -> Int
forall a. a -> Maybe a -> a
fromMaybe Int
start (Text -> Map Text Int -> Maybe Int
forall k a. Ord k => k -> Map k a -> Maybe a
M.lookup Text
prefix Map Text Int
i))))
  where
    start :: Int
start = Int
1
    prefix :: Text
prefix = Int -> Text -> Text
T.take Int
20 Text
rendered
    rendered :: Text
rendered =
      case NameTemplate
template of
        ArrayRelationTemplate Text
sample -> Text
"ar_" Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
sample
        ArrayAggregateTemplate Text
sample -> Text
"aa_" Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
sample
        ObjectRelationTemplate Text
sample -> Text
"or_" Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
sample
        TableTemplate Text
sample -> Text
"t_" Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
sample
        ForOrderAlias Text
sample -> Text
"order_" Text -> Text -> Text
forall a. Semigroup a => a -> a -> a
<> Text
sample
        NameTemplate
IndexTemplate -> Text
"idx"
        NameTemplate
UnnestTemplate -> Text
"unnest"
        FunctionTemplate FunctionName {Maybe Text
Text
$sel:functionName:FunctionName :: FunctionName -> Text
functionName :: Text
functionNameSchema :: Maybe Text
$sel:functionNameSchema:FunctionName :: FunctionName -> Maybe Text
..} -> Text
functionName
        NativeQueryTemplate NativeQueryName {Name
getNativeQueryName :: Name
getNativeQueryName :: NativeQueryName -> Name
..} -> Name -> Text
G.unName Name
getNativeQueryName

fromAlias :: From -> EntityAlias
fromAlias :: From -> EntityAlias
fromAlias (FromQualifiedTable Aliased {Text
$sel:aliasedAlias:Aliased :: forall a. Aliased a -> Text
aliasedAlias :: Text
aliasedAlias}) = Text -> EntityAlias
EntityAlias Text
aliasedAlias
fromAlias (FromSelect Aliased {Text
$sel:aliasedAlias:Aliased :: forall a. Aliased a -> Text
aliasedAlias :: Text
aliasedAlias}) = Text -> EntityAlias
EntityAlias Text
aliasedAlias
fromAlias (FromSelectJson Aliased {Text
$sel:aliasedAlias:Aliased :: forall a. Aliased a -> Text
aliasedAlias :: Text
aliasedAlias}) = Text -> EntityAlias
EntityAlias Text
aliasedAlias
fromAlias (FromFunction Aliased {Text
$sel:aliasedAlias:Aliased :: forall a. Aliased a -> Text
aliasedAlias :: Text
aliasedAlias}) = Text -> EntityAlias
EntityAlias Text
aliasedAlias
fromAlias (FromNativeQuery Aliased {Text
$sel:aliasedAlias:Aliased :: forall a. Aliased a -> Text
aliasedAlias :: Text
aliasedAlias}) = Text -> EntityAlias
EntityAlias Text
aliasedAlias

fieldTextNames :: Ir.AnnFieldsG 'BigQuery Void Expression -> [Text]
fieldTextNames :: Fields (AnnFieldG 'BigQuery Void Expression) -> [Text]
fieldTextNames = ((FieldName, AnnFieldG 'BigQuery Void Expression) -> Text)
-> Fields (AnnFieldG 'BigQuery Void Expression) -> [Text]
forall a b. (a -> b) -> [a] -> [b]
forall (f :: * -> *) a b. Functor f => (a -> b) -> f a -> f b
fmap (\(Rql.FieldName Text
name, AnnFieldG 'BigQuery Void Expression
_) -> Text
name)

unEntityAlias :: EntityAlias -> Text
unEntityAlias :: EntityAlias -> Text
unEntityAlias (EntityAlias Text
t) = Text
t

--------------------------------------------------------------------------------
-- Global limit support

getGlobalTop :: FromIr Top
getGlobalTop :: FromIr Top
getGlobalTop =
  ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  Top
-> FromIr Top
forall a.
ReaderT
  FromIrReader
  (StateT
     FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
  a
-> FromIr a
FromIr
    ( (FromIrReader -> Top)
-> ReaderT
     FromIrReader
     (StateT
        FromIrState (WriterT FromIrWriter (Validate (NonEmpty Error))))
     Top
forall r (m :: * -> *) a. MonadReader r m => (r -> a) -> m a
asks
        ( \FromIrReader {config :: FromIrReader -> FromIrConfig
config = FromIrConfig {Top
globalSelectLimit :: FromIrConfig -> Top
globalSelectLimit :: Top
globalSelectLimit}} ->
            Top
globalSelectLimit
        )
    )