You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@calcite.apache.org by "Dominik Labuda (Jira)" <ji...@apache.org> on 2021/02/08 18:53:00 UTC

[jira] [Updated] (CALCITE-4491) Aggregation of window function produces invalid SQL for PostgreSQL

     [ https://issues.apache.org/jira/browse/CALCITE-4491?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Dominik Labuda updated CALCITE-4491:
------------------------------------
    Description: 
h2. Issue

We tested the following behavior against PostgreSQL database, however more of the supported dialects may be affected.

When aggregating the results of a window function an invalid SQL is generated. I was able to replicate the behavior in tests with:
{code:java}
@Test void testAggregatingWindowFunction() {
    final RelBuilder builder = relBuilder();
    final RelNode root = builder
        .scan("EMP")
        .project(
            builder.alias(
                builder.getRexBuilder().makeOver(
                    builder.getTypeFactory().createSqlType(SqlTypeName.INTEGER),
                    SqlStdOperatorTable.RANK,
                    new ArrayList<>(),
                    new ArrayList<>(),
                    ImmutableList.of(new RexFieldCollation(builder.field("SAL"), ImmutableSet.of())),
                    RexWindowBounds.UNBOUNDED_PRECEDING,
                    RexWindowBounds.UNBOUNDED_FOLLOWING,
                    true,
                    true,
                    false,
                    false,
                    false
                ),
                "rank"
            )
        )
        .as("tmp")
        .aggregate(
            builder.groupKey(),
            builder.count(
                true,
                "cnt",
                builder.field("tmp", "rank")
            )
        )
        .build();
    final String expectedSql = "SELECT COUNT(DISTINCT \"rank\") AS \"cnt\"\n" +
        "FROM (SELECT RANK() OVER (ORDER BY \"SAL\") AS \"rank\"\n" +
        "FROM \"scott\".\"EMP\") AS \"t\"";
    assertThat(
        toSql(root, PostgresqlSqlDialect.DEFAULT),
        isLinux(expectedSql)
    );
  }
{code}
The code above fails, since it produces this SQL instead of the expected one, which cannot be executed on PGSQL:
{noformat}
SELECT COUNT(DISTINCT RANK() OVER (ORDER BY "SAL")) AS "cnt"
FROM "scott"."EMP"
{noformat}
In that case I am getting this kinds of errors from DB:
{noformat}
ERROR: aggregate function calls cannot contain window function calls
{noformat}

h2. Suggested solution

Since _SqlDialect_ already contains support for determining whether the databases support nested aggregations via _public boolean supportsNestedAggregations()_ we could either add another method like _public boolean supportsNestedWindows()_ maybe _supportNestedWindowsInAggregation()_ to be more verbose. Or we could reuse the existing method for the window purposes (which seems non-transparent). Then we will be able to handle the behavior in _SqlImplementor.needNewSubQuery()_ similarly as it already does with the aggregations.

Please let me know if you agree and I will be more than happy to provide you with PR to review, thanks!

  was:
h2. Issue

We tested the following behavior against PostgreSQL database, however more of the supported dialects may be affected.

When aggregating the results of a window function an invalid SQL is generated. I was able to replicate the behavior in tests with:
{code:java}
@Test void testAggregatingWindowFunction() {
    final RelBuilder builder = relBuilder();
    final RelNode root = builder
        .scan("EMP")
        .project(
            builder.alias(
                builder.getRexBuilder().makeOver(
                    builder.getTypeFactory().createSqlType(SqlTypeName.INTEGER),
                    SqlStdOperatorTable.RANK,
                    new ArrayList<>(),
                    new ArrayList<>(),
                    ImmutableList.of(new RexFieldCollation(builder.field("SAL"), ImmutableSet.of())),
                    RexWindowBounds.UNBOUNDED_PRECEDING,
                    RexWindowBounds.UNBOUNDED_FOLLOWING,
                    true,
                    true,
                    false,
                    false,
                    false
                ),
                "rank"
            )
        )
        .as("tmp")
        .aggregate(
            builder.groupKey(),
            builder.count(
                true,
                "cnt",
                builder.field("tmp", "rank")
            )
        )
        .build();
    final String expectedSql = "SELECT COUNT(DISTINCT \"rank\") AS \"cnt\"\n" +
        "FROM (SELECT RANK() OVER (ORDER BY \"SAL\") AS \"rank\"\n" +
        "FROM \"scott\".\"EMP\") AS \"t\"";
    assertThat(
        toSql(root, PostgresqlSqlDialect.DEFAULT),
        isLinux(expectedSql)
    );
  }
{code}
The code above fails, since it produces this SQL instead of the expected one, which cannot be executed on PGSQL:
{noformat}
SELECT COUNT(DISTINCT RANK() OVER (ORDER BY "SAL")) AS "cnt"
FROM "scott"."EMP"
{noformat}
In that case I am getting this kinds of errors from DB:
{noformat}
ERROR: aggregate function calls cannot contain window function calls
{noformat}

h2. Suggested solution

Since _SqlDialect_ already contains support for determining whether the databases support nested aggregations via _public boolean supportsNestedAggregations()_ we could either add another method like _public boolean supportsNestedWindows()_ maybe _supportNestedWindowsInAggregation_ to be more verbose. Or we could reuse the existing method for the window purposes (which seems non-transparent). Then we will be able to handle the behavior in _SqlImplementor.needNewSubQuery()_ similarly as it already does with the aggregations.

Please let me know if you agree and I will be more than happy to provide you with PR to review, thanks!


> Aggregation of window function produces invalid SQL for PostgreSQL
> ------------------------------------------------------------------
>
>                 Key: CALCITE-4491
>                 URL: https://issues.apache.org/jira/browse/CALCITE-4491
>             Project: Calcite
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 1.26.0
>            Reporter: Dominik Labuda
>            Priority: Major
>
> h2. Issue
> We tested the following behavior against PostgreSQL database, however more of the supported dialects may be affected.
> When aggregating the results of a window function an invalid SQL is generated. I was able to replicate the behavior in tests with:
> {code:java}
> @Test void testAggregatingWindowFunction() {
>     final RelBuilder builder = relBuilder();
>     final RelNode root = builder
>         .scan("EMP")
>         .project(
>             builder.alias(
>                 builder.getRexBuilder().makeOver(
>                     builder.getTypeFactory().createSqlType(SqlTypeName.INTEGER),
>                     SqlStdOperatorTable.RANK,
>                     new ArrayList<>(),
>                     new ArrayList<>(),
>                     ImmutableList.of(new RexFieldCollation(builder.field("SAL"), ImmutableSet.of())),
>                     RexWindowBounds.UNBOUNDED_PRECEDING,
>                     RexWindowBounds.UNBOUNDED_FOLLOWING,
>                     true,
>                     true,
>                     false,
>                     false,
>                     false
>                 ),
>                 "rank"
>             )
>         )
>         .as("tmp")
>         .aggregate(
>             builder.groupKey(),
>             builder.count(
>                 true,
>                 "cnt",
>                 builder.field("tmp", "rank")
>             )
>         )
>         .build();
>     final String expectedSql = "SELECT COUNT(DISTINCT \"rank\") AS \"cnt\"\n" +
>         "FROM (SELECT RANK() OVER (ORDER BY \"SAL\") AS \"rank\"\n" +
>         "FROM \"scott\".\"EMP\") AS \"t\"";
>     assertThat(
>         toSql(root, PostgresqlSqlDialect.DEFAULT),
>         isLinux(expectedSql)
>     );
>   }
> {code}
> The code above fails, since it produces this SQL instead of the expected one, which cannot be executed on PGSQL:
> {noformat}
> SELECT COUNT(DISTINCT RANK() OVER (ORDER BY "SAL")) AS "cnt"
> FROM "scott"."EMP"
> {noformat}
> In that case I am getting this kinds of errors from DB:
> {noformat}
> ERROR: aggregate function calls cannot contain window function calls
> {noformat}
> h2. Suggested solution
> Since _SqlDialect_ already contains support for determining whether the databases support nested aggregations via _public boolean supportsNestedAggregations()_ we could either add another method like _public boolean supportsNestedWindows()_ maybe _supportNestedWindowsInAggregation()_ to be more verbose. Or we could reuse the existing method for the window purposes (which seems non-transparent). Then we will be able to handle the behavior in _SqlImplementor.needNewSubQuery()_ similarly as it already does with the aggregations.
> Please let me know if you agree and I will be more than happy to provide you with PR to review, thanks!



--
This message was sent by Atlassian Jira
(v8.3.4#803005)