Rui, I think I found another potential bug with rank().
+--------+--------+ |column_1|column_2| +--------+--------+ |1 |100 | |1 |200 | +--------+--------+ Query using Beam SQL: SELECT *, RANK() over (PARTITION BY column_1 ORDER BY column_2 DESC) AS agg FROM PCOLLECTION Result: [1, 200, 2] [1, 100, 1] While I expect the result to be: [1, 200, 1] [1, 100, 2] So basically the rank result (by using desc order) seems incorrect to me. Can you please take a look at this issue? Thanks! From: Tao Li <t...@zillow.com> Reply-To: "user@beam.apache.org" <user@beam.apache.org> Date: Friday, March 5, 2021 at 1:37 PM To: "user@beam.apache.org" <user@beam.apache.org>, Rui Wang <ruw...@google.com> Subject: Re: Regarding the over window query support from Beam SQL Hi Rui, Yes that’s the problem. The alias is not propagated to the final schema. Created https://issues.apache.org/jira/browse/BEAM-11930 Thanks! From: Rui Wang <amaliu...@apache.org> Reply-To: "user@beam.apache.org" <user@beam.apache.org> Date: Friday, March 5, 2021 at 11:31 AM To: user <user@beam.apache.org> Subject: Re: Regarding the over window query support from Beam SQL I see. So the problem is the alias does appear in the output schema? Based on your log: the logical plan contains the "agg" as alias but the physical plan (the BeamWindowRel) seems not showing the alias. I think it's worth opening a JIRA now to further investigate why the alias did not correctly pass through. The entry point is to investigate from BeamWindowRel. -Rui On Fri, Mar 5, 2021 at 10:20 AM Tao Li <t...@zillow.com<mailto:t...@zillow.com>> wrote: Hi Rui, Just following up on this issue. Do you think this is a bug? Is there a workaround? Thanks! From: Tao Li <t...@zillow.com<mailto:t...@zillow.com>> Reply-To: "user@beam.apache.org<mailto:user@beam.apache.org>" <user@beam.apache.org<mailto:user@beam.apache.org>> Date: Tuesday, March 2, 2021 at 3:37 PM To: Rui Wang <amaliu...@apache.org<mailto:amaliu...@apache.org>> Cc: "user@beam.apache.org<mailto:user@beam.apache.org>" <user@beam.apache.org<mailto:user@beam.apache.org>> Subject: Re: Regarding the over window query support from Beam SQL Hi Rui, Thanks for this info. It’s good to know we are already supporting the window function. But I still have a problem with the schema of the query result. This is my code (with Beam 2.28): Schema appSchema = Schema .builder() .addInt32Field("foo") .addInt32Field("bar") .build(); Row rowOne = Row.withSchema(appSchema).addValues(1, 1).build(); Row rowTwo = Row.withSchema(appSchema).addValues(1, 2).build(); PCollection<Row> inputRows = executionContext.getPipeline() .apply(Create.of(rowOne, rowTwo)) .setRowSchema(appSchema); String sql = "SELECT foo, bar, RANK() over (PARTITION BY foo ORDER BY bar) AS agg FROM PCOLLECTION"; PCollection<Row> result = inputRows.apply("sql", SqlTransform.query(sql)); I can see the expected data from result, but I don’t see “agg” column in the schema. Do you have any ideas regarding this issue? Thanks! The Beam schema of the result is: Field{name=foo, description=, type=FieldType{typeName=INT32, nullable=false, logicalType=null, collectionElementType=null, mapKeyType=null, mapValueType=null, rowSchema=null, metadata={}}, options={{}}} Field{name=bar, description=, type=FieldType{typeName=INT32, nullable=false, logicalType=null, collectionElementType=null, mapKeyType=null, mapValueType=null, rowSchema=null, metadata={}}, options={{}}} Field{name=w0$o0, description=, type=FieldType{typeName=INT64, nullable=false, logicalType=null, collectionElementType=null, mapKeyType=null, mapValueType=null, rowSchema=null, metadata={}}, options={{}}} Here are some detailed logs if they are helpful: [main] INFO org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner - SQL: SELECT `PCOLLECTION`.`foo`, `PCOLLECTION`.`bar`, RANK() OVER (PARTITION BY `PCOLLECTION`.`foo` ORDER BY `PCOLLECTION`.`bar`) AS `agg` FROM `beam`.`PCOLLECTION` AS `PCOLLECTION` [main] INFO org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner - SQLPlan> LogicalProject(foo=[$0], bar=[$1], agg=[RANK() OVER (PARTITION BY $0 ORDER BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) BeamIOSourceRel(table=[[beam, PCOLLECTION]]) [main] INFO org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner - BEAMPlan> BeamWindowRel(window#0=[window(partition {0} order by [1] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [RANK()])]) BeamIOSourceRel(table=[[beam, PCOLLECTION]]) From: Rui Wang <amaliu...@apache.org<mailto:amaliu...@apache.org>> Date: Tuesday, March 2, 2021 at 10:43 AM To: Tao Li <t...@zillow.com<mailto:t...@zillow.com>> Cc: "user@beam.apache.org<mailto:user@beam.apache.org>" <user@beam.apache.org<mailto:user@beam.apache.org>> Subject: Re: Regarding the over window query support from Beam SQL Hi Tao, [1] contains what functions are working with OVER clause. Rank is one of the functions that is supported. Can you take a look? [1]: https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamAnalyticFunctionsTest.java<https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fapache%2Fbeam%2Fblob%2Fmaster%2Fsdks%2Fjava%2Fextensions%2Fsql%2Fsrc%2Ftest%2Fjava%2Forg%2Fapache%2Fbeam%2Fsdk%2Fextensions%2Fsql%2FBeamAnalyticFunctionsTest.java&data=04%7C01%7Ctaol%40zillow.com%7Cb4a68902ee214a99e2be08d8e00d4c47%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637505695016240771%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=cK%2BrOS0wvVFduMQ74FHRCeDzvpxRhwtKMUD341NGFc8%3D&reserved=0> -Rui On Tue, Mar 2, 2021 at 9:24 AM Tao Li <t...@zillow.com<mailto:t...@zillow.com>> wrote: + Rui Wang. Looks like Rui has been working on this jira. From: Tao Li <t...@zillow.com<mailto:t...@zillow.com>> Date: Monday, March 1, 2021 at 9:51 PM To: "user@beam.apache.org<mailto:user@beam.apache.org>" <user@beam.apache.org<mailto:user@beam.apache.org>> Subject: Regarding the over window query support from Beam SQL Hi Beam community, Querying over a window for ranking etc is pretty common in SQL use cases. I have found this jira https://issues.apache.org/jira/browse/BEAM-9198<https://nam11.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FBEAM-9198&data=04%7C01%7Ctaol%40zillow.com%7Cb4a68902ee214a99e2be08d8e00d4c47%7C033464830d1840e7a5883784ac50e16f%7C0%7C0%7C637505695016250731%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000&sdata=vW%2BWjopv2tIXB%2FPyGfDFkOYu7zUOMz%2BZJ3BsurpsZ3Q%3D&reserved=0> Do we have a plan to support this? If there is no such plan in near future, are Beam developers supposed to implement this function on their own (e.g. by using GroupBy)? Thanks!