[
https://issues.apache.org/jira/browse/BEAM-12097?focusedWorklogId=753784&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-753784
]
ASF GitHub Bot logged work on BEAM-12097:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 06/Apr/22 23:50
Start Date: 06/Apr/22 23:50
Worklog Time Spent: 10m
Work Description: apilloud commented on PR #16911:
URL: https://github.com/apache/beam/pull/16911#issuecomment-1090934645
The question is essentially what is the purpose of the ZetaSQL translation
library. `AggregateScan` is a SQL query with a group by for example `select
value*2, sum(key+1) from KeyValue group by value` This would give a groupByList
of [value*2] and aggregate_list of [sum(key+1)]. The LogicalProject is the
first step of simplifying the data, it basicly translates to a simple ParDo. We
take the input data and apply any simple transforms, so you get: LogicalProject
[value*2, key+1]. The input to the logical project may have other columns that
are dropped. After the logical project you are left with a simpler SQL query:
`SELECT col1, sum(col2) from LogicalProject group by col1`.
In AnalyticScan, it appears the equivalent is `functionGroupList`.
Issue Time Tracking
-------------------
Worklog Id: (was: 753784)
Time Spent: 1.5h (was: 1h 20m)
> OVER clause converted from ZetaSQL to Logical SQL plan
> ------------------------------------------------------
>
> Key: BEAM-12097
> URL: https://issues.apache.org/jira/browse/BEAM-12097
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql-zetasql
> Reporter: Andrew Pilloud
> Assignee: Mike Hernandez
> Priority: P3
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> The OVER clause isn't supported by our ZetaSQL to Calcite translator. It can
> be trivially enabled in the parser with the example below, but there is some
> work required to convert the parsed ZetaSQL proto to Calcite logical
> operators (mostly in AggregateScanConverter).
> This is the "over clause" TODO here:
> https://github.com/apache/beam/blob/243128a8fc52798e1b58b0cf1a271d95ee7aa241/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java#L147
> {code:java}
> a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
> +++
> b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
> @@ -144,6 +144,7 @@ public class SqlAnalyzer {
> .setEnabledLanguageFeatures(
> new HashSet<>(
> Arrays.asList(
> + LanguageFeature.FEATURE_ANALYTIC_FUNCTIONS,
> LanguageFeature.FEATURE_NUMERIC_TYPE,
> LanguageFeature.FEATURE_DISALLOW_GROUP_BY_FLOAT,
> LanguageFeature.FEATURE_V_1_2_CIVIL_TIME,
> diff --git
> a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
>
> b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
> index 33889f34884..fd107ac5721 100644
> ---
> a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
> +++
> b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
> @@ -3461,6 +3461,16 @@ public class ZetaSqlDialectSpecTest extends
> ZetaSqlTestBase {
> zetaSQLQueryPlanner.convertToBeamRel(sql);
> }
>
> + @Test
> + public void testAnalyticOver() {
> + String sql = "select sum(Key) over () From KeyValue";
> +
> + ZetaSQLQueryPlanner zetaSQLQueryPlanner = new
> ZetaSQLQueryPlanner(config);
> + thrown.expect(UnsupportedOperationException.class);
> + thrown.expectMessage("Does not support sub-queries");
> + zetaSQLQueryPlanner.convertToBeamRel(sql);
> + }
> +
> @Test
> public void testSubstr() {
> String sql = "SELECT substr(@p0, @p1, @p2)"; {code}
> Current state the test fails:
> {code}
> java.lang.UnsupportedOperationException: Conversion of RESOLVED_ANALYTIC_SCAN
> is not supported
> at
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.getConverterRule(QueryStatementConverter.java:108
> )
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:99)
>
> at
> java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
>
> at
> java.base/java.util.Collections$2.tryAdvance(Collections.java:4756)
>
> at
> java.base/java.util.Collections$2.forEachRemaining(Collections.java:4764)
>
> at
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
>
> at
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
>
> at
> java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
>
> at
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>
> at
> java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:101)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convert(QueryStatementConverter.java:89)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertRootQuery(QueryStatementConverter.java:55)
> at
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLPlannerImpl.rel(ZetaSQLPlannerImpl.java:98)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRelInternal(ZetaSQLQueryPlanner.java:313)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:301)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:285)
>
> at
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlDialectSpecTest.testAnalyticOver(ZetaSqlDialectSpecTest.java:3471)
>
> {code}
--
This message was sent by Atlassian Jira
(v8.20.1#820001)