mirror of
https://github.com/hasura/graphql-engine.git
synced 2025-01-05 22:34:22 +03:00
distinct_on for BigQuery
PR-URL: https://github.com/hasura/graphql-engine-mono/pull/3191 GitOrigin-RevId: e9f1c814b2caf09946389c4bcb30a0c42277abaf
This commit is contained in:
parent
bd8766171f
commit
7beb15b4a3
@ -8,6 +8,7 @@
|
|||||||
- server: in mssql transactions, rollback only if the transaction is active
|
- server: in mssql transactions, rollback only if the transaction is active
|
||||||
- server: add request and response bodies to OpenAPI specification of REST endpoints
|
- server: add request and response bodies to OpenAPI specification of REST endpoints
|
||||||
- server: implement upsert mutations for MS SQL Server (close #7864)
|
- server: implement upsert mutations for MS SQL Server (close #7864)
|
||||||
|
- server: bigquery: implement `distinct_on`.
|
||||||
|
|
||||||
### Bug fixes and improvements
|
### Bug fixes and improvements
|
||||||
(Add entries below in the order of server, console, cli, docs, others)
|
(Add entries below in the order of server, console, cli, docs, others)
|
||||||
|
@ -12,6 +12,7 @@ import Data.Aeson qualified as J
|
|||||||
import Data.ByteString.Lazy qualified as L
|
import Data.ByteString.Lazy qualified as L
|
||||||
import Data.Environment qualified as Env
|
import Data.Environment qualified as Env
|
||||||
import Data.HashMap.Strict qualified as HM
|
import Data.HashMap.Strict qualified as HM
|
||||||
|
import Data.Int qualified as Int
|
||||||
import Data.Text qualified as T
|
import Data.Text qualified as T
|
||||||
import Data.Text.Encoding qualified as T
|
import Data.Text.Encoding qualified as T
|
||||||
import Data.Time.Clock.System
|
import Data.Time.Clock.System
|
||||||
@ -28,7 +29,7 @@ import Hasura.RQL.Types.SourceCustomization
|
|||||||
import Hasura.RQL.Types.Table
|
import Hasura.RQL.Types.Table
|
||||||
import Hasura.SQL.Backend
|
import Hasura.SQL.Backend
|
||||||
|
|
||||||
defaultGlobalSelectLimit :: Int
|
defaultGlobalSelectLimit :: Int.Int64
|
||||||
defaultGlobalSelectLimit = 1000
|
defaultGlobalSelectLimit = 1000
|
||||||
|
|
||||||
resolveSourceConfig ::
|
resolveSourceConfig ::
|
||||||
|
@ -16,10 +16,10 @@ where
|
|||||||
|
|
||||||
import Control.Monad.Validate
|
import Control.Monad.Validate
|
||||||
import Data.HashMap.Strict qualified as HM
|
import Data.HashMap.Strict qualified as HM
|
||||||
|
import Data.Int qualified as Int
|
||||||
import Data.List.NonEmpty qualified as NE
|
import Data.List.NonEmpty qualified as NE
|
||||||
import Data.Map.Strict (Map)
|
import Data.Map.Strict (Map)
|
||||||
import Data.Map.Strict qualified as M
|
import Data.Map.Strict qualified as M
|
||||||
import Data.Proxy
|
|
||||||
import Data.Text qualified as T
|
import Data.Text qualified as T
|
||||||
import Hasura.Backends.BigQuery.Instances.Types ()
|
import Hasura.Backends.BigQuery.Instances.Types ()
|
||||||
import Hasura.Backends.BigQuery.Source (BigQuerySourceConfig (..))
|
import Hasura.Backends.BigQuery.Source (BigQuerySourceConfig (..))
|
||||||
@ -49,7 +49,6 @@ data Error
|
|||||||
| UnsupportedSQLExp Expression
|
| UnsupportedSQLExp Expression
|
||||||
| UnsupportedDistinctOn
|
| UnsupportedDistinctOn
|
||||||
| InvalidIntegerishSql Expression
|
| InvalidIntegerishSql Expression
|
||||||
| DistinctIsn'tSupported
|
|
||||||
| ConnectionsNotSupported
|
| ConnectionsNotSupported
|
||||||
| ActionsNotSupported
|
| ActionsNotSupported
|
||||||
|
|
||||||
@ -69,7 +68,6 @@ instance Show Error where
|
|||||||
UnsupportedSQLExp {} -> "UnsupportedSQLExp"
|
UnsupportedSQLExp {} -> "UnsupportedSQLExp"
|
||||||
UnsupportedDistinctOn {} -> "UnsupportedDistinctOn"
|
UnsupportedDistinctOn {} -> "UnsupportedDistinctOn"
|
||||||
InvalidIntegerishSql {} -> "InvalidIntegerishSql"
|
InvalidIntegerishSql {} -> "InvalidIntegerishSql"
|
||||||
DistinctIsn'tSupported {} -> "DistinctIsn'tSupported"
|
|
||||||
ConnectionsNotSupported {} -> "ConnectionsNotSupported"
|
ConnectionsNotSupported {} -> "ConnectionsNotSupported"
|
||||||
ActionsNotSupported {} -> "ActionsNotSupported"
|
ActionsNotSupported {} -> "ActionsNotSupported"
|
||||||
|
|
||||||
@ -140,7 +138,7 @@ mkSQLSelect ::
|
|||||||
Ir.AnnSelectG 'BigQuery Void (Ir.AnnFieldG 'BigQuery Void) Expression ->
|
Ir.AnnSelectG 'BigQuery Void (Ir.AnnFieldG 'BigQuery Void) Expression ->
|
||||||
FromIr BigQuery.Select
|
FromIr BigQuery.Select
|
||||||
mkSQLSelect jsonAggSelect annSimpleSel = do
|
mkSQLSelect jsonAggSelect annSimpleSel = do
|
||||||
select <- fromSelectRows annSimpleSel
|
select <- noExtraPartitionFields <$> fromSelectRows annSimpleSel
|
||||||
pure
|
pure
|
||||||
( select
|
( select
|
||||||
{ selectCardinality =
|
{ selectCardinality =
|
||||||
@ -177,8 +175,22 @@ fromUnnestedJSON json columns _fields = do
|
|||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
fromSelectRows :: Ir.AnnSelectG 'BigQuery Void (Ir.AnnFieldG 'BigQuery Void) Expression -> FromIr BigQuery.Select
|
fromSelectRows :: Ir.AnnSelectG 'BigQuery Void (Ir.AnnFieldG 'BigQuery Void) Expression -> FromIr BigQuery.PartitionableSelect
|
||||||
fromSelectRows annSelectG = do
|
fromSelectRows annSelectG = do
|
||||||
|
let Ir.AnnSelectG
|
||||||
|
{ _asnFields = fields,
|
||||||
|
_asnFrom = from,
|
||||||
|
_asnPerm = perm,
|
||||||
|
_asnArgs = args,
|
||||||
|
_asnStrfyNum = num
|
||||||
|
} = annSelectG
|
||||||
|
Ir.TablePerm {_tpLimit = mPermLimit, _tpFilter = permFilter} = perm
|
||||||
|
permissionBasedTop =
|
||||||
|
maybe NoTop (Top . fromIntegral) mPermLimit
|
||||||
|
stringifyNumbers =
|
||||||
|
if num
|
||||||
|
then StringifyNumbers
|
||||||
|
else LeaveNumbersAlone
|
||||||
selectFrom <-
|
selectFrom <-
|
||||||
case from of
|
case from of
|
||||||
Ir.FromTable qualifiedObject -> fromQualifiedTable qualifiedObject
|
Ir.FromTable qualifiedObject -> fromQualifiedTable qualifiedObject
|
||||||
@ -190,7 +202,7 @@ fromSelectRows annSelectG = do
|
|||||||
argsWhere,
|
argsWhere,
|
||||||
argsJoins,
|
argsJoins,
|
||||||
argsTop,
|
argsTop,
|
||||||
argsDistinct = Proxy,
|
argsDistinct,
|
||||||
argsOffset,
|
argsOffset,
|
||||||
argsExistingJoins
|
argsExistingJoins
|
||||||
} <-
|
} <-
|
||||||
@ -205,35 +217,119 @@ fromSelectRows annSelectG = do
|
|||||||
NE.nonEmpty (concatMap (toList . fieldSourceProjections True) fieldSources)
|
NE.nonEmpty (concatMap (toList . fieldSourceProjections True) fieldSources)
|
||||||
`onNothing` refute (pure NoProjectionFields)
|
`onNothing` refute (pure NoProjectionFields)
|
||||||
globalTop <- getGlobalTop
|
globalTop <- getGlobalTop
|
||||||
|
let select =
|
||||||
|
Select
|
||||||
|
{ selectCardinality = Many,
|
||||||
|
selectFinalWantedFields = pure (fieldTextNames fields),
|
||||||
|
selectGroupBy = mempty,
|
||||||
|
selectOrderBy = argsOrderBy,
|
||||||
|
-- We DO APPLY the global top here, because this pulls down all rows.
|
||||||
|
selectTop = globalTop <> permissionBasedTop <> argsTop,
|
||||||
|
selectProjections,
|
||||||
|
selectFrom,
|
||||||
|
selectJoins = argsJoins <> concat (mapMaybe fieldSourceJoins fieldSources),
|
||||||
|
selectWhere = argsWhere <> Where [filterExpression],
|
||||||
|
selectOffset = int64Expr <$> argsOffset
|
||||||
|
}
|
||||||
|
case argsDistinct of
|
||||||
|
Nothing ->
|
||||||
|
pure $ simpleSelect select
|
||||||
|
Just distinct ->
|
||||||
|
simulateDistinctOn select distinct 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 distinctOnColumns orderByColumns = do
|
||||||
|
rowNumAlias <- generateEntityAlias IndexTemplate
|
||||||
pure
|
pure
|
||||||
Select
|
PartitionableSelect
|
||||||
{ selectCardinality = Many,
|
{ pselectFrom = selectFrom select,
|
||||||
selectFinalWantedFields = pure (fieldTextNames fields),
|
pselectFinalize = \mExtraPartitionField ->
|
||||||
selectGroupBy = mempty,
|
let -- we use the same alias both for outer and inner selects
|
||||||
selectOrderBy = argsOrderBy,
|
alias = entityAliasText (fromAlias (selectFrom select))
|
||||||
-- We DO APPLY the global top here, because this pulls down all rows.
|
distinctFields = fmap (\(ColumnName name) -> FieldName name alias) distinctOnColumns
|
||||||
selectTop = globalTop <> permissionBasedTop <> argsTop,
|
finalDistinctFields = case mExtraPartitionField of
|
||||||
selectProjections,
|
Just extraFields
|
||||||
selectFrom,
|
| Just neExtraFields <- nonEmpty extraFields ->
|
||||||
selectJoins = argsJoins <> concat (mapMaybe fieldSourceJoins fieldSources),
|
neExtraFields <> distinctFields
|
||||||
selectWhere = argsWhere <> Where [filterExpression],
|
_ -> distinctFields
|
||||||
selectOffset = argsOffset
|
(distinctOnOrderBy, innerOrderBy) =
|
||||||
|
case orderByColumns of
|
||||||
|
Just orderBy ->
|
||||||
|
let (distincts, others) = NE.partition (\OrderBy {..} -> orderByFieldName `elem` distinctFields) orderBy
|
||||||
|
in (NE.nonEmpty distincts, NE.nonEmpty others)
|
||||||
|
Nothing ->
|
||||||
|
(Nothing, Nothing)
|
||||||
|
innerFrom =
|
||||||
|
FromSelect
|
||||||
|
Aliased
|
||||||
|
{ aliasedAlias = alias,
|
||||||
|
aliasedThing =
|
||||||
|
select
|
||||||
|
{ selectProjections =
|
||||||
|
StarProjection
|
||||||
|
:| [ WindowProjection
|
||||||
|
( Aliased
|
||||||
|
{ aliasedAlias = unEntityAlias rowNumAlias,
|
||||||
|
aliasedThing =
|
||||||
|
RowNumberOverPartitionBy
|
||||||
|
finalDistinctFields
|
||||||
|
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
|
||||||
|
}
|
||||||
|
)
|
||||||
|
],
|
||||||
|
selectTop = mempty,
|
||||||
|
selectJoins = mempty,
|
||||||
|
selectOrderBy = mempty,
|
||||||
|
selectOffset = Nothing,
|
||||||
|
selectGroupBy = mempty,
|
||||||
|
selectFinalWantedFields = mempty
|
||||||
|
}
|
||||||
|
}
|
||||||
|
in select
|
||||||
|
{ selectFrom = innerFrom,
|
||||||
|
selectWhere =
|
||||||
|
Where
|
||||||
|
[ EqualExpression
|
||||||
|
(ColumnExpression FieldName {fieldNameEntity = alias, fieldName = unEntityAlias rowNumAlias})
|
||||||
|
(int64Expr 1)
|
||||||
|
],
|
||||||
|
selectOrderBy = distinctOnOrderBy
|
||||||
|
}
|
||||||
}
|
}
|
||||||
where
|
|
||||||
Ir.AnnSelectG
|
|
||||||
{ _asnFields = fields,
|
|
||||||
_asnFrom = from,
|
|
||||||
_asnPerm = perm,
|
|
||||||
_asnArgs = args,
|
|
||||||
_asnStrfyNum = num
|
|
||||||
} = annSelectG
|
|
||||||
Ir.TablePerm {_tpLimit = mPermLimit, _tpFilter = permFilter} = perm
|
|
||||||
permissionBasedTop =
|
|
||||||
maybe NoTop Top mPermLimit
|
|
||||||
stringifyNumbers =
|
|
||||||
if num
|
|
||||||
then StringifyNumbers
|
|
||||||
else LeaveNumbersAlone
|
|
||||||
|
|
||||||
fromSelectAggregate ::
|
fromSelectAggregate ::
|
||||||
Maybe (EntityAlias, HashMap ColumnName ColumnName) ->
|
Maybe (EntityAlias, HashMap ColumnName ColumnName) ->
|
||||||
@ -244,7 +340,7 @@ fromSelectAggregate minnerJoinFields annSelectG = do
|
|||||||
case from of
|
case from of
|
||||||
Ir.FromTable qualifiedObject -> fromQualifiedTable qualifiedObject
|
Ir.FromTable qualifiedObject -> fromQualifiedTable qualifiedObject
|
||||||
_ -> refute (pure (FromTypeUnsupported from))
|
_ -> refute (pure (FromTypeUnsupported from))
|
||||||
args'@Args {argsWhere, argsOrderBy, argsJoins, argsTop, argsOffset, argsDistinct = Proxy} <-
|
args'@Args {argsWhere, argsOrderBy, argsJoins, argsTop, argsOffset, argsDistinct} <-
|
||||||
runReaderT (fromSelectArgsG args) (fromAlias selectFrom)
|
runReaderT (fromSelectArgsG args) (fromAlias selectFrom)
|
||||||
filterExpression <-
|
filterExpression <-
|
||||||
runReaderT (fromAnnBoolExp permFilter) (fromAlias selectFrom)
|
runReaderT (fromAnnBoolExp permFilter) (fromAlias selectFrom)
|
||||||
@ -271,6 +367,45 @@ fromSelectAggregate minnerJoinFields annSelectG = do
|
|||||||
)
|
)
|
||||||
(refute (pure NoProjectionFields))
|
(refute (pure NoProjectionFields))
|
||||||
indexAlias <- generateEntityAlias IndexTemplate
|
indexAlias <- generateEntityAlias IndexTemplate
|
||||||
|
let innerSelectAlias = entityAliasText (fromAlias selectFrom)
|
||||||
|
mDistinctFields = fmap (fmap (\(ColumnName name) -> FieldName name innerSelectAlias)) argsDistinct
|
||||||
|
mPartitionFields =
|
||||||
|
fmap (NE.fromList . map fst) mforeignKeyConditions <> mDistinctFields
|
||||||
|
innerProjections =
|
||||||
|
case mPartitionFields of
|
||||||
|
Nothing -> pure StarProjection
|
||||||
|
Just partitionFields ->
|
||||||
|
StarProjection
|
||||||
|
:|
|
||||||
|
-- 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.
|
||||||
|
[ WindowProjection
|
||||||
|
( Aliased
|
||||||
|
{ aliasedAlias = unEntityAlias indexAlias,
|
||||||
|
aliasedThing =
|
||||||
|
RowNumberOverPartitionBy
|
||||||
|
-- The row numbers start from 1.
|
||||||
|
partitionFields
|
||||||
|
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 =
|
||||||
|
ColumnExpression $
|
||||||
|
FieldName
|
||||||
|
{ fieldNameEntity = innerSelectAlias,
|
||||||
|
fieldName = unEntityAlias indexAlias
|
||||||
|
}
|
||||||
pure
|
pure
|
||||||
Select
|
Select
|
||||||
{ selectCardinality = One,
|
{ selectCardinality = One,
|
||||||
@ -283,36 +418,7 @@ fromSelectAggregate minnerJoinFields annSelectG = do
|
|||||||
( Aliased
|
( Aliased
|
||||||
{ aliasedThing =
|
{ aliasedThing =
|
||||||
Select
|
Select
|
||||||
{ selectProjections =
|
{ selectProjections = innerProjections,
|
||||||
case mforeignKeyConditions of
|
|
||||||
Nothing -> pure StarProjection
|
|
||||||
Just innerJoinFields ->
|
|
||||||
pure StarProjection
|
|
||||||
<>
|
|
||||||
-- 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.
|
|
||||||
pure
|
|
||||||
( WindowProjection
|
|
||||||
( Aliased
|
|
||||||
{ aliasedAlias = unEntityAlias indexAlias,
|
|
||||||
aliasedThing =
|
|
||||||
RowNumberOverPartitionBy
|
|
||||||
-- The row numbers start from 1.
|
|
||||||
(NE.fromList (map fst innerJoinFields))
|
|
||||||
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!
|
|
||||||
}
|
|
||||||
)
|
|
||||||
),
|
|
||||||
selectFrom,
|
selectFrom,
|
||||||
selectJoins = argsJoins,
|
selectJoins = argsJoins,
|
||||||
selectWhere = argsWhere <> (Where [filterExpression]),
|
selectWhere = argsWhere <> (Where [filterExpression]),
|
||||||
@ -325,55 +431,51 @@ fromSelectAggregate minnerJoinFields annSelectG = do
|
|||||||
selectFinalWantedFields = Nothing,
|
selectFinalWantedFields = Nothing,
|
||||||
selectCardinality = Many,
|
selectCardinality = Many,
|
||||||
selectTop = maybe argsTop (const NoTop) mforeignKeyConditions,
|
selectTop = maybe argsTop (const NoTop) mforeignKeyConditions,
|
||||||
selectOffset = maybe argsOffset (const Nothing) mforeignKeyConditions,
|
-- we apply offset only if we don't have partitions
|
||||||
|
-- when we do OFFSET/LIMIT based on ROW_NUMBER()
|
||||||
|
selectOffset = maybe (int64Expr <$> argsOffset) (const Nothing) mPartitionFields,
|
||||||
selectGroupBy = mempty
|
selectGroupBy = mempty
|
||||||
},
|
},
|
||||||
aliasedAlias = entityAliasText (fromAlias selectFrom)
|
aliasedAlias = innerSelectAlias
|
||||||
}
|
}
|
||||||
),
|
),
|
||||||
selectJoins = concat (mapMaybe fieldSourceJoins fieldSources),
|
selectJoins = concat (mapMaybe fieldSourceJoins fieldSources),
|
||||||
selectWhere =
|
selectWhere =
|
||||||
case mforeignKeyConditions of
|
case mPartitionFields of
|
||||||
Nothing -> mempty
|
Nothing -> mempty
|
||||||
Just {} ->
|
Just {} ->
|
||||||
let offset =
|
let offset =
|
||||||
case argsOffset of
|
case argsDistinct of
|
||||||
Nothing -> mempty
|
Nothing ->
|
||||||
Just offset' ->
|
case argsOffset of
|
||||||
Where
|
Nothing -> mempty
|
||||||
-- Apply an offset using the row_number from above.
|
Just offset' ->
|
||||||
[ OpExpression
|
-- Apply an offset using the row_number from above.
|
||||||
MoreOp
|
[ OpExpression
|
||||||
( ColumnExpression
|
MoreOp
|
||||||
FieldName
|
indexColumn
|
||||||
{ fieldNameEntity =
|
(int64Expr offset')
|
||||||
coerce (fromAlias selectFrom),
|
]
|
||||||
fieldName = unEntityAlias indexAlias
|
Just {} ->
|
||||||
}
|
-- in case of distinct_on we need to select the row number offset+1
|
||||||
)
|
-- effectively skipping number of rows equal to offset
|
||||||
offset'
|
[ EqualExpression
|
||||||
]
|
indexColumn
|
||||||
|
(int64Expr (fromMaybe 0 argsOffset + 1))
|
||||||
|
]
|
||||||
limit =
|
limit =
|
||||||
case argsTop of
|
case argsTop of
|
||||||
NoTop -> mempty
|
NoTop -> mempty
|
||||||
Top limit' ->
|
Top limit' ->
|
||||||
Where
|
-- Apply a limit using the row_number from above.
|
||||||
-- Apply a limit using the row_number from above.
|
[ OpExpression
|
||||||
[ OpExpression
|
LessOp
|
||||||
LessOp
|
indexColumn
|
||||||
( ColumnExpression
|
( int64Expr (limit' + 1) -- Because the row_number() indexing starts at 1.
|
||||||
FieldName
|
-- So idx<l+1 means idx<2 where l = 1 i.e. "limit to 1 row".
|
||||||
{ fieldNameEntity =
|
)
|
||||||
coerce (fromAlias selectFrom),
|
]
|
||||||
fieldName = unEntityAlias indexAlias
|
in Where (offset <> limit),
|
||||||
}
|
|
||||||
)
|
|
||||||
( ValueExpression . IntegerValue . Int64 . tshow $
|
|
||||||
limit' + 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 offset <> limit,
|
|
||||||
selectOrderBy = Nothing,
|
selectOrderBy = Nothing,
|
||||||
selectOffset = Nothing
|
selectOffset = Nothing
|
||||||
}
|
}
|
||||||
@ -387,7 +489,7 @@ fromSelectAggregate minnerJoinFields annSelectG = do
|
|||||||
} = annSelectG
|
} = annSelectG
|
||||||
Ir.TablePerm {_tpLimit = mPermLimit, _tpFilter = permFilter} = perm
|
Ir.TablePerm {_tpLimit = mPermLimit, _tpFilter = permFilter} = perm
|
||||||
permissionBasedTop =
|
permissionBasedTop =
|
||||||
maybe NoTop Top mPermLimit
|
maybe NoTop (Top . fromIntegral) mPermLimit
|
||||||
stringifyNumbers =
|
stringifyNumbers =
|
||||||
if num
|
if num
|
||||||
then StringifyNumbers
|
then StringifyNumbers
|
||||||
@ -401,8 +503,8 @@ data Args = Args
|
|||||||
argsOrderBy :: Maybe (NonEmpty OrderBy),
|
argsOrderBy :: Maybe (NonEmpty OrderBy),
|
||||||
argsJoins :: [Join],
|
argsJoins :: [Join],
|
||||||
argsTop :: Top,
|
argsTop :: Top,
|
||||||
argsOffset :: Maybe Expression,
|
argsOffset :: Maybe Int.Int64,
|
||||||
argsDistinct :: Proxy (Maybe (NonEmpty FieldName)),
|
argsDistinct :: Maybe (NonEmpty ColumnName),
|
||||||
argsExistingJoins :: Map TableName EntityAlias
|
argsExistingJoins :: Map TableName EntityAlias
|
||||||
}
|
}
|
||||||
deriving (Show)
|
deriving (Show)
|
||||||
@ -416,20 +518,9 @@ data UnfurledJoin = UnfurledJoin
|
|||||||
|
|
||||||
fromSelectArgsG :: Ir.SelectArgsG 'BigQuery Expression -> ReaderT EntityAlias FromIr Args
|
fromSelectArgsG :: Ir.SelectArgsG 'BigQuery Expression -> ReaderT EntityAlias FromIr Args
|
||||||
fromSelectArgsG selectArgsG = do
|
fromSelectArgsG selectArgsG = do
|
||||||
let argsOffset = ValueExpression . IntegerValue . Int64 . tshow <$> moffset
|
|
||||||
argsWhere <-
|
argsWhere <-
|
||||||
maybe (pure mempty) (fmap (Where . pure) . fromAnnBoolExp) mannBoolExp
|
maybe (pure mempty) (fmap (Where . pure) . fromAnnBoolExp) mannBoolExp
|
||||||
argsTop <- maybe (pure mempty) (pure . Top) mlimit
|
let argsTop = maybe mempty (Top . fromIntegral) mlimit
|
||||||
-- Not supported presently, per Vamshi:
|
|
||||||
--
|
|
||||||
-- > It is hardly used and we don't have to go to great lengths to support it.
|
|
||||||
--
|
|
||||||
-- But placeholdering the code so that when it's ready to be used,
|
|
||||||
-- you can just drop the Proxy wrapper.
|
|
||||||
argsDistinct <-
|
|
||||||
case mdistinct of
|
|
||||||
Nothing -> pure Proxy
|
|
||||||
Just {} -> refute (pure DistinctIsn'tSupported)
|
|
||||||
(argsOrderBy, joins) <-
|
(argsOrderBy, joins) <-
|
||||||
runWriterT (traverse fromAnnotatedOrderByItemG (maybe [] toList orders))
|
runWriterT (traverse fromAnnotatedOrderByItemG (maybe [] toList orders))
|
||||||
-- Any object-relation joins that we generated, we record their
|
-- Any object-relation joins that we generated, we record their
|
||||||
@ -440,13 +531,14 @@ fromSelectArgsG selectArgsG = do
|
|||||||
Args
|
Args
|
||||||
{ argsJoins = toList (fmap unfurledJoin joins),
|
{ argsJoins = toList (fmap unfurledJoin joins),
|
||||||
argsOrderBy = NE.nonEmpty argsOrderBy,
|
argsOrderBy = NE.nonEmpty argsOrderBy,
|
||||||
|
argsDistinct = mdistinct,
|
||||||
..
|
..
|
||||||
}
|
}
|
||||||
where
|
where
|
||||||
Ir.SelectArgs
|
Ir.SelectArgs
|
||||||
{ _saWhere = mannBoolExp,
|
{ _saWhere = mannBoolExp,
|
||||||
_saLimit = mlimit,
|
_saLimit = mlimit,
|
||||||
_saOffset = moffset,
|
_saOffset = argsOffset,
|
||||||
_saDistinct = mdistinct,
|
_saDistinct = mdistinct,
|
||||||
_saOrderBy = orders
|
_saOrderBy = orders
|
||||||
} = selectArgsG
|
} = selectArgsG
|
||||||
@ -1213,17 +1305,22 @@ fromArrayAggregateSelectG annRelationSelectG = do
|
|||||||
--
|
--
|
||||||
-- ORDER BY artist_other_id;
|
-- ORDER BY artist_other_id;
|
||||||
-- ^ Ordering for the artist table should appear here.
|
-- ^ 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 ::
|
fromArrayRelationSelectG ::
|
||||||
Ir.ArrayRelationSelectG 'BigQuery Void Expression ->
|
Ir.ArrayRelationSelectG 'BigQuery Void Expression ->
|
||||||
ReaderT EntityAlias FromIr Join
|
ReaderT EntityAlias FromIr Join
|
||||||
fromArrayRelationSelectG annRelationSelectG = do
|
fromArrayRelationSelectG annRelationSelectG = do
|
||||||
select <- lift (fromSelectRows annSelectG) -- Take the original select.
|
pselect <- lift (fromSelectRows annSelectG) -- Take the original select.
|
||||||
joinFieldName <- lift (fromRelName aarRelationshipName)
|
joinFieldName <- lift (fromRelName aarRelationshipName)
|
||||||
alias <- lift (generateEntityAlias (ArrayRelationTemplate joinFieldName))
|
alias <- lift (generateEntityAlias (ArrayRelationTemplate joinFieldName))
|
||||||
indexAlias <- lift (generateEntityAlias IndexTemplate)
|
indexAlias <- lift (generateEntityAlias IndexTemplate)
|
||||||
joinOn <- fromMappingFieldNames alias mapping
|
joinOn <- fromMappingFieldNames alias mapping
|
||||||
innerJoinFields <-
|
innerJoinFields <-
|
||||||
fromMappingFieldNames (fromAlias (selectFrom select)) mapping
|
fromMappingFieldNames (fromAlias (pselectFrom pselect)) mapping
|
||||||
|
let select = withExtraPartitionFields pselect $ map fst innerJoinFields
|
||||||
let joinFieldProjections =
|
let joinFieldProjections =
|
||||||
map
|
map
|
||||||
( \(fieldName', _) ->
|
( \(fieldName', _) ->
|
||||||
|
@ -20,7 +20,6 @@ import Hasura.GraphQL.Schema.Build qualified as GSB
|
|||||||
import Hasura.GraphQL.Schema.Common
|
import Hasura.GraphQL.Schema.Common
|
||||||
import Hasura.GraphQL.Schema.Select
|
import Hasura.GraphQL.Schema.Select
|
||||||
import Hasura.Prelude
|
import Hasura.Prelude
|
||||||
import Hasura.RQL.IR
|
|
||||||
import Hasura.RQL.IR.Select qualified as IR
|
import Hasura.RQL.IR.Select qualified as IR
|
||||||
import Hasura.RQL.Types
|
import Hasura.RQL.Types
|
||||||
import Language.GraphQL.Draft.Syntax qualified as G
|
import Language.GraphQL.Draft.Syntax qualified as G
|
||||||
@ -44,7 +43,7 @@ instance BackendSchema 'BigQuery where
|
|||||||
nodesAggExtension = Just ()
|
nodesAggExtension = Just ()
|
||||||
|
|
||||||
-- table arguments
|
-- table arguments
|
||||||
tableArguments = bqTableArgs
|
tableArguments = defaultTableArgs
|
||||||
|
|
||||||
-- indivdual components
|
-- indivdual components
|
||||||
columnParser = bqColumnParser
|
columnParser = bqColumnParser
|
||||||
@ -145,34 +144,6 @@ bqBuildFunctionMutationFields ::
|
|||||||
bqBuildFunctionMutationFields _ _ _ _ _ =
|
bqBuildFunctionMutationFields _ _ _ _ _ =
|
||||||
pure []
|
pure []
|
||||||
|
|
||||||
----------------------------------------------------------------
|
|
||||||
-- Table arguments
|
|
||||||
|
|
||||||
bqTableArgs ::
|
|
||||||
forall r m n.
|
|
||||||
MonadBuildSchema 'BigQuery r m n =>
|
|
||||||
SourceName ->
|
|
||||||
TableInfo 'BigQuery ->
|
|
||||||
SelPermInfo 'BigQuery ->
|
|
||||||
m (InputFieldsParser n (IR.SelectArgsG 'BigQuery (UnpreparedValue 'BigQuery)))
|
|
||||||
bqTableArgs sourceName tableInfo selectPermissions = do
|
|
||||||
whereParser <- tableWhereArg sourceName tableInfo selectPermissions
|
|
||||||
orderByParser <- tableOrderByArg sourceName tableInfo selectPermissions
|
|
||||||
pure do
|
|
||||||
whereArg <- whereParser
|
|
||||||
orderByArg <- orderByParser
|
|
||||||
limitArg <- tableLimitArg
|
|
||||||
offsetArg <- tableOffsetArg
|
|
||||||
pure $
|
|
||||||
IR.SelectArgs
|
|
||||||
{ IR._saWhere = whereArg,
|
|
||||||
IR._saOrderBy = orderByArg,
|
|
||||||
IR._saLimit = limitArg,
|
|
||||||
IR._saOffset = offsetArg,
|
|
||||||
-- not supported on BigQuery for now
|
|
||||||
IR._saDistinct = Nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
-- Individual components
|
-- Individual components
|
||||||
|
|
||||||
|
@ -22,6 +22,7 @@ import Data.Aeson.Casing qualified as J
|
|||||||
import Data.Aeson.TH qualified as J
|
import Data.Aeson.TH qualified as J
|
||||||
import Data.ByteString.Lazy qualified as BL
|
import Data.ByteString.Lazy qualified as BL
|
||||||
import Data.HashMap.Strict qualified as HM
|
import Data.HashMap.Strict qualified as HM
|
||||||
|
import Data.Int qualified as Int
|
||||||
import Data.Text.Encoding qualified as TE
|
import Data.Text.Encoding qualified as TE
|
||||||
import Data.X509 qualified as X509
|
import Data.X509 qualified as X509
|
||||||
import Data.X509.Memory qualified as X509
|
import Data.X509.Memory qualified as X509
|
||||||
@ -160,7 +161,7 @@ data BigQuerySourceConfig = BigQuerySourceConfig
|
|||||||
_scDatasets :: ![Text],
|
_scDatasets :: ![Text],
|
||||||
_scProjectId :: !Text, -- this is part of service-account.json, but we put it here on purpose
|
_scProjectId :: !Text, -- this is part of service-account.json, but we put it here on purpose
|
||||||
_scAccessTokenMVar :: !(MVar (Maybe TokenResp)),
|
_scAccessTokenMVar :: !(MVar (Maybe TokenResp)),
|
||||||
_scGlobalSelectLimit :: !Int
|
_scGlobalSelectLimit :: !Int.Int64
|
||||||
}
|
}
|
||||||
deriving (Eq)
|
deriving (Eq)
|
||||||
|
|
||||||
|
@ -34,6 +34,10 @@ module Hasura.Backends.BigQuery.Types
|
|||||||
Reselect (..),
|
Reselect (..),
|
||||||
ScalarType (..),
|
ScalarType (..),
|
||||||
Select (..),
|
Select (..),
|
||||||
|
PartitionableSelect (..),
|
||||||
|
noExtraPartitionFields,
|
||||||
|
withExtraPartitionFields,
|
||||||
|
simpleSelect,
|
||||||
SelectJson (..),
|
SelectJson (..),
|
||||||
TableName (..),
|
TableName (..),
|
||||||
Time (..),
|
Time (..),
|
||||||
@ -48,6 +52,7 @@ module Hasura.Backends.BigQuery.Types
|
|||||||
doubleToFloat64,
|
doubleToFloat64,
|
||||||
getGQLTableName,
|
getGQLTableName,
|
||||||
intToInt64,
|
intToInt64,
|
||||||
|
int64Expr,
|
||||||
isComparableType,
|
isComparableType,
|
||||||
isNumType,
|
isNumType,
|
||||||
parseScalarValue,
|
parseScalarValue,
|
||||||
@ -65,6 +70,7 @@ import Data.ByteString (ByteString)
|
|||||||
import Data.ByteString.Base64 qualified as Base64
|
import Data.ByteString.Base64 qualified as Base64
|
||||||
import Data.ByteString.Lazy qualified as L
|
import Data.ByteString.Lazy qualified as L
|
||||||
import Data.Coerce
|
import Data.Coerce
|
||||||
|
import Data.Int qualified as Int
|
||||||
import Data.Scientific
|
import Data.Scientific
|
||||||
import Data.Text qualified as T
|
import Data.Text qualified as T
|
||||||
import Data.Text.Encoding qualified as T
|
import Data.Text.Encoding qualified as T
|
||||||
@ -105,6 +111,29 @@ instance Cacheable Select
|
|||||||
|
|
||||||
instance NFData Select
|
instance NFData Select
|
||||||
|
|
||||||
|
-- | Helper type allowing addition of extra fields used
|
||||||
|
-- in PARTITION BY.
|
||||||
|
--
|
||||||
|
-- The main purpose of this type is sumulation of DISTINCT ON
|
||||||
|
-- implemented in Hasura.Backends.BigQuery.FromIr.simulateDistinctOn
|
||||||
|
data PartitionableSelect = PartitionableSelect
|
||||||
|
{ pselectFinalize :: Maybe [FieldName] -> Select,
|
||||||
|
pselectFrom :: !From
|
||||||
|
}
|
||||||
|
|
||||||
|
simpleSelect :: Select -> PartitionableSelect
|
||||||
|
simpleSelect select =
|
||||||
|
PartitionableSelect
|
||||||
|
{ pselectFinalize = const select,
|
||||||
|
pselectFrom = selectFrom select
|
||||||
|
}
|
||||||
|
|
||||||
|
noExtraPartitionFields :: PartitionableSelect -> Select
|
||||||
|
noExtraPartitionFields PartitionableSelect {..} = pselectFinalize Nothing
|
||||||
|
|
||||||
|
withExtraPartitionFields :: PartitionableSelect -> [FieldName] -> Select
|
||||||
|
withExtraPartitionFields PartitionableSelect {..} = pselectFinalize . Just
|
||||||
|
|
||||||
data ArrayAgg = ArrayAgg
|
data ArrayAgg = ArrayAgg
|
||||||
{ arrayAggProjections :: !(NonEmpty Projection),
|
{ arrayAggProjections :: !(NonEmpty Projection),
|
||||||
arrayAggOrderBy :: !(Maybe (NonEmpty OrderBy)),
|
arrayAggOrderBy :: !(Maybe (NonEmpty OrderBy)),
|
||||||
@ -327,7 +356,7 @@ instance NFData AsStruct
|
|||||||
|
|
||||||
data Top
|
data Top
|
||||||
= NoTop
|
= NoTop
|
||||||
| Top Int
|
| Top Int.Int64
|
||||||
deriving (Eq, Ord, Show, Generic, Data, Lift)
|
deriving (Eq, Ord, Show, Generic, Data, Lift)
|
||||||
|
|
||||||
instance FromJSON Top
|
instance FromJSON Top
|
||||||
@ -678,9 +707,12 @@ instance FromJSON Int64 where parseJSON = liberalInt64Parser Int64
|
|||||||
|
|
||||||
instance ToJSON Int64 where toJSON = liberalIntegralPrinter
|
instance ToJSON Int64 where toJSON = liberalIntegralPrinter
|
||||||
|
|
||||||
intToInt64 :: Int -> Int64
|
intToInt64 :: Int.Int64 -> Int64
|
||||||
intToInt64 = Int64 . tshow
|
intToInt64 = Int64 . tshow
|
||||||
|
|
||||||
|
int64Expr :: Int.Int64 -> Expression
|
||||||
|
int64Expr = ValueExpression . IntegerValue . intToInt64
|
||||||
|
|
||||||
-- | BigQuery's conception of a fixed precision decimal.
|
-- | BigQuery's conception of a fixed precision decimal.
|
||||||
newtype Decimal = Decimal Text
|
newtype Decimal = Decimal Text
|
||||||
deriving (Show, Eq, Ord, Generic, Data, Cacheable, NFData, Hashable, Lift)
|
deriving (Show, Eq, Ord, Generic, Data, Cacheable, NFData, Hashable, Lift)
|
||||||
|
@ -184,6 +184,8 @@ instance Cacheable Int where unchanged _ = (==)
|
|||||||
|
|
||||||
instance Cacheable Int32 where unchanged _ = (==)
|
instance Cacheable Int32 where unchanged _ = (==)
|
||||||
|
|
||||||
|
instance Cacheable Int64 where unchanged _ = (==)
|
||||||
|
|
||||||
instance Cacheable Integer where unchanged _ = (==)
|
instance Cacheable Integer where unchanged _ = (==)
|
||||||
|
|
||||||
instance Cacheable Scientific where unchanged _ = (==)
|
instance Cacheable Scientific where unchanged _ = (==)
|
||||||
|
@ -50,6 +50,28 @@
|
|||||||
author:
|
author:
|
||||||
id: '2'
|
id: '2'
|
||||||
name: Author 2
|
name: Author 2
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
author:
|
||||||
|
id: '2'
|
||||||
|
name: Author 2
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
author:
|
||||||
|
id: '2'
|
||||||
|
name: Author 2
|
||||||
|
articles_aggregate:
|
||||||
|
nodes:
|
||||||
|
- id: '3'
|
||||||
|
title: Title 3
|
||||||
|
author:
|
||||||
|
id: '2'
|
||||||
|
name: Author 2
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
author:
|
||||||
|
id: '2'
|
||||||
|
name: Author 2
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
@ -96,6 +118,8 @@
|
|||||||
nodes:
|
nodes:
|
||||||
- title: Title 3
|
- title: Title 3
|
||||||
article_citations: []
|
article_citations: []
|
||||||
|
- title: Title 4
|
||||||
|
article_citations: []
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
|
@ -22,6 +22,12 @@
|
|||||||
author:
|
author:
|
||||||
id: '2'
|
id: '2'
|
||||||
name: Author 2
|
name: Author 2
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
|
author:
|
||||||
|
id: '2'
|
||||||
|
name: Author 2
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
@ -57,6 +63,9 @@
|
|||||||
- id: '3'
|
- id: '3'
|
||||||
title: Title 3
|
title: Title 3
|
||||||
content: Content 3
|
content: Content 3
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
|
175
server/tests-py/queries/graphql_query/bigquery/distinct_on.yaml
Normal file
175
server/tests-py/queries/graphql_query/bigquery/distinct_on.yaml
Normal file
@ -0,0 +1,175 @@
|
|||||||
|
- description: oldest articles by author, distinct on is_published with author and count of citations
|
||||||
|
url: /v1/graphql
|
||||||
|
status: 200
|
||||||
|
response:
|
||||||
|
data:
|
||||||
|
hasura_test_article:
|
||||||
|
- id: '1'
|
||||||
|
title: Title 1
|
||||||
|
created_at: '2008-12-25T07:30:01Z'
|
||||||
|
is_published: false
|
||||||
|
author:
|
||||||
|
name: Author 1
|
||||||
|
article_citations_aggregate:
|
||||||
|
aggregate:
|
||||||
|
count: '2'
|
||||||
|
- id: '2'
|
||||||
|
title: Title 2
|
||||||
|
created_at: '2008-12-25T07:30:02Z'
|
||||||
|
is_published: true
|
||||||
|
author:
|
||||||
|
name: Author 1
|
||||||
|
article_citations_aggregate:
|
||||||
|
aggregate:
|
||||||
|
count: '1'
|
||||||
|
- id: '3'
|
||||||
|
title: Title 3
|
||||||
|
created_at: '2008-12-25T07:30:03Z'
|
||||||
|
is_published: false
|
||||||
|
author:
|
||||||
|
name: Author 2
|
||||||
|
article_citations_aggregate:
|
||||||
|
aggregate:
|
||||||
|
count: '0'
|
||||||
|
query:
|
||||||
|
query: |
|
||||||
|
query {
|
||||||
|
hasura_test_article(distinct_on: [author_id, is_published], order_by: [{author_id:asc}, {is_published: asc}, {created_at: asc}]) {
|
||||||
|
id
|
||||||
|
title
|
||||||
|
created_at
|
||||||
|
is_published
|
||||||
|
author {
|
||||||
|
name
|
||||||
|
}
|
||||||
|
article_citations_aggregate {
|
||||||
|
aggregate {
|
||||||
|
count
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
- description: author with oldest articles, distinct on is_published, with sums of ids of those articles
|
||||||
|
url: /v1/graphql
|
||||||
|
status: 200
|
||||||
|
response:
|
||||||
|
data:
|
||||||
|
hasura_test_author:
|
||||||
|
- id: '1'
|
||||||
|
name: Author 1
|
||||||
|
articles:
|
||||||
|
- title: Title 1
|
||||||
|
id: '1'
|
||||||
|
- title: Title 2
|
||||||
|
id: '2'
|
||||||
|
articles_aggregate:
|
||||||
|
aggregate:
|
||||||
|
sum:
|
||||||
|
id: '3'
|
||||||
|
- id: '2'
|
||||||
|
name: Author 2
|
||||||
|
articles:
|
||||||
|
- title: Title 3
|
||||||
|
id: '3'
|
||||||
|
articles_aggregate:
|
||||||
|
aggregate:
|
||||||
|
sum:
|
||||||
|
id: '3'
|
||||||
|
query:
|
||||||
|
query: |
|
||||||
|
query {
|
||||||
|
hasura_test_author(order_by: [{id:asc}]) {
|
||||||
|
id
|
||||||
|
name
|
||||||
|
articles(distinct_on: is_published, order_by: [{is_published: asc}, {created_at: asc}]) {
|
||||||
|
title
|
||||||
|
id
|
||||||
|
}
|
||||||
|
articles_aggregate(distinct_on: is_published, order_by: [{is_published: asc}, {created_at: asc}]) {
|
||||||
|
aggregate {
|
||||||
|
sum {
|
||||||
|
id
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
- description: author with newest articles, distinct on is_published, with sums of ids of those articles
|
||||||
|
url: /v1/graphql
|
||||||
|
status: 200
|
||||||
|
response:
|
||||||
|
data:
|
||||||
|
hasura_test_author:
|
||||||
|
- id: '1'
|
||||||
|
name: Author 1
|
||||||
|
articles:
|
||||||
|
- title: Title 1
|
||||||
|
id: '1'
|
||||||
|
- title: Title 2
|
||||||
|
id: '2'
|
||||||
|
articles_aggregate:
|
||||||
|
aggregate:
|
||||||
|
sum:
|
||||||
|
id: '3'
|
||||||
|
- id: '2'
|
||||||
|
name: Author 2
|
||||||
|
articles:
|
||||||
|
- title: Title 4
|
||||||
|
id: '4'
|
||||||
|
articles_aggregate:
|
||||||
|
aggregate:
|
||||||
|
sum:
|
||||||
|
id: '4'
|
||||||
|
query:
|
||||||
|
query: |
|
||||||
|
query {
|
||||||
|
hasura_test_author(order_by: [{id:asc}]) {
|
||||||
|
id
|
||||||
|
name
|
||||||
|
articles(distinct_on: is_published, order_by: [{is_published: asc}, {created_at: desc}]) {
|
||||||
|
title
|
||||||
|
id
|
||||||
|
}
|
||||||
|
articles_aggregate(distinct_on: is_published, order_by: [{is_published: asc}, {created_at: desc}]) {
|
||||||
|
aggregate {
|
||||||
|
sum {
|
||||||
|
id
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
- description: first and last articles distinct by author and published status
|
||||||
|
url: /v1/graphql
|
||||||
|
status: 200
|
||||||
|
response:
|
||||||
|
data:
|
||||||
|
first:
|
||||||
|
- id: '1'
|
||||||
|
title: Title 1
|
||||||
|
- id: '2'
|
||||||
|
title: Title 2
|
||||||
|
- id: '3'
|
||||||
|
title: Title 3
|
||||||
|
last:
|
||||||
|
- id: '1'
|
||||||
|
title: Title 1
|
||||||
|
- id: '2'
|
||||||
|
title: Title 2
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
query:
|
||||||
|
query: |
|
||||||
|
query {
|
||||||
|
first: hasura_test_article(distinct_on: [author_id, is_published], order_by: [{author_id: asc}, {is_published: asc}, {created_at: asc}]) {
|
||||||
|
id
|
||||||
|
title
|
||||||
|
}
|
||||||
|
last: hasura_test_article(distinct_on: [author_id, is_published], order_by: [{author_id: asc}, {is_published: asc}, {created_at: desc}]) {
|
||||||
|
id
|
||||||
|
title
|
||||||
|
}
|
||||||
|
}
|
@ -37,6 +37,10 @@
|
|||||||
title: Title 3
|
title: Title 3
|
||||||
content: Content 3
|
content: Content 3
|
||||||
article_citations: []
|
article_citations: []
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
|
article_citations: []
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
@ -89,6 +93,12 @@
|
|||||||
article_citations_aggregate:
|
article_citations_aggregate:
|
||||||
aggregate:
|
aggregate:
|
||||||
count: '0'
|
count: '0'
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
|
article_citations_aggregate:
|
||||||
|
aggregate:
|
||||||
|
count: '0'
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
@ -135,6 +145,10 @@
|
|||||||
title: Title 3
|
title: Title 3
|
||||||
content: Content 3
|
content: Content 3
|
||||||
article_citations2: []
|
article_citations2: []
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
|
article_citations2: []
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
@ -145,7 +159,7 @@
|
|||||||
id
|
id
|
||||||
title
|
title
|
||||||
content
|
content
|
||||||
article_citations2 {
|
article_citations2 (order_by: {cited_article_id: asc}) {
|
||||||
description
|
description
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -22,6 +22,12 @@ response:
|
|||||||
author:
|
author:
|
||||||
id: '2'
|
id: '2'
|
||||||
name: Author 2
|
name: Author 2
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
|
author:
|
||||||
|
id: '2'
|
||||||
|
name: Author 2
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
|
@ -66,6 +66,9 @@ args:
|
|||||||
INSERT INTO `hasura_test.article` VALUES (
|
INSERT INTO `hasura_test.article` VALUES (
|
||||||
3, "Title 3", "Content 3", 2, FALSE, NULL, PARSE_TIMESTAMP("%c", "Thu Dec 25 07:30:03 2008")
|
3, "Title 3", "Content 3", 2, FALSE, NULL, PARSE_TIMESTAMP("%c", "Thu Dec 25 07:30:03 2008")
|
||||||
);
|
);
|
||||||
|
INSERT INTO `hasura_test.article` VALUES (
|
||||||
|
4, "Title 4", "Content 4", 2, FALSE, NULL, PARSE_TIMESTAMP("%c", "Thu Dec 26 07:31:04 2008")
|
||||||
|
);
|
||||||
|
|
||||||
-- a copy for remote joins as we can't have the same table tracked in two sources
|
-- a copy for remote joins as we can't have the same table tracked in two sources
|
||||||
CREATE TABLE `hasura_test.article2`
|
CREATE TABLE `hasura_test.article2`
|
||||||
|
@ -7,6 +7,7 @@ response:
|
|||||||
- id: '1'
|
- id: '1'
|
||||||
- id: '2'
|
- id: '2'
|
||||||
- id: '3'
|
- id: '3'
|
||||||
|
- id: '4'
|
||||||
- data:
|
- data:
|
||||||
hasura_test_author:
|
hasura_test_author:
|
||||||
- id: '1'
|
- id: '1'
|
||||||
|
@ -37,3 +37,5 @@ response:
|
|||||||
articles:
|
articles:
|
||||||
- title: Title 3
|
- title: Title 3
|
||||||
id: '3'
|
id: '3'
|
||||||
|
- title: Title 4
|
||||||
|
id: '4'
|
||||||
|
@ -46,3 +46,6 @@ response:
|
|||||||
- title: Title 3
|
- title: Title 3
|
||||||
author:
|
author:
|
||||||
name: Author 2
|
name: Author 2
|
||||||
|
- title: Title 4
|
||||||
|
author:
|
||||||
|
name: Author 2
|
||||||
|
@ -15,6 +15,9 @@
|
|||||||
- id: '3'
|
- id: '3'
|
||||||
title: Title 3
|
title: Title 3
|
||||||
content: Content 3
|
content: Content 3
|
||||||
|
- id: '4'
|
||||||
|
title: Title 4
|
||||||
|
content: Content 4
|
||||||
query:
|
query:
|
||||||
query: |
|
query: |
|
||||||
query {
|
query {
|
||||||
|
@ -216,6 +216,9 @@ class TestGraphQLQueryBasicBigquery:
|
|||||||
def test_agg_nodes(self, hge_ctx, transport):
|
def test_agg_nodes(self, hge_ctx, transport):
|
||||||
check_query_f(hge_ctx, self.dir() + "/agg_nodes.yaml", transport)
|
check_query_f(hge_ctx, self.dir() + "/agg_nodes.yaml", transport)
|
||||||
|
|
||||||
|
def test_distinct_on(self, hge_ctx, transport):
|
||||||
|
check_query_f(hge_ctx, self.dir() + "/distinct_on.yaml", transport)
|
||||||
|
|
||||||
@classmethod
|
@classmethod
|
||||||
def dir(cls):
|
def dir(cls):
|
||||||
return 'queries/graphql_query/bigquery'
|
return 'queries/graphql_query/bigquery'
|
||||||
|
Loading…
Reference in New Issue
Block a user