[
https://issues.apache.org/jira/browse/BEAM-12097?focusedWorklogId=747473&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-747473
]
ASF GitHub Bot logged work on BEAM-12097:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Mar/22 21:54
Start Date: 24/Mar/22 21:54
Worklog Time Spent: 10m
Work Description: roger-mike commented on pull request #16911:
URL: https://github.com/apache/beam/pull/16911#issuecomment-1078420044
Hi @apilloud, thanks for the advice. As it is now, I'm getting the following
error when running `testAnalyticOver`. The implementation is still incomplete
but wanted to ask you if you have an idea of why this is happening. Thanks a
lot.
```
index (3) must be less than size (3)
java.lang.IndexOutOfBoundsException: index (3) must be less than size (3)
at
org.apache.beam.vendor.calcite.v1_28_0.com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:1404)
at
org.apache.beam.vendor.calcite.v1_28_0.com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:1386)
at
org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.RegularImmutableList.get(RegularImmutableList.java:77)
at
org.apache.beam.sdk.extensions.sql.zetasql.translation.ExpressionConverter.retrieveRexNode(ExpressionConverter.java:213)
at
org.apache.beam.sdk.extensions.sql.zetasql.translation.ProjectScanConverter.convert(ProjectScanConverter.java:46)
at
org.apache.beam.sdk.extensions.sql.zetasql.translation.ProjectScanConverter.convert(ProjectScanConverter.java:30)
at
org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:91)
at
org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convert(QueryStatementConverter.java:78)
at
org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertRootQuery(QueryStatementConverter.java:43)
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:198)
at
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:186)
at
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:170)
at
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlDialectSpecTest.testAnalyticOver(ZetaSqlDialectSpecTest.java:3582)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
at
org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:323)
at
org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:258)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
at
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
at
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
at
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
at
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
at
org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
at
org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
at
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
at
org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
at
org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
at
org.gradle.api.internal.tasks.testing.worker.TestWorker$2.run(TestWorker.java:176)
at
org.gradle.api.internal.tasks.testing.worker.TestWorker.executeAndMaintainThreadName(TestWorker.java:129)
at
org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:100)
at
org.gradle.api.internal.tasks.testing.worker.TestWorker.execute(TestWorker.java:60)
at
org.gradle.process.internal.worker.child.ActionExecutionWorker.execute(ActionExecutionWorker.java:56)
at
org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:133)
at
org.gradle.process.internal.worker.child.SystemApplicationClassLoaderWorker.call(SystemApplicationClassLoaderWorker.java:71)
at
worker.org.gradle.process.internal.worker.GradleWorkerMain.run(GradleWorkerMain.java:69)
at
worker.org.gradle.process.internal.worker.GradleWorkerMain.main(GradleWorkerMain.java:74)
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 747473)
Time Spent: 1h (was: 50m)
> 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: 1h
> 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)