+d...@beam.apache.org <d...@beam.apache.org> I tried reproducing this but was not successful, the output schema was as expected. I added the following to BeamSqlMultipleSchemasTest.java at head. (I did discover that PAssert.that(result).containsInAnyOrder(output) doesn't validate column names however.)
@Test public void testSelectAs() { PCollection<Row> input = pipeline.apply(create(row(1, "strstr"))); PCollection<Row> result = input.apply(SqlTransform.query("WITH tempTable (id, v) AS (SELECT f_int as id, f_string as v FROM PCOLLECTION) SELECT id AS fout_int, v AS fout_string FROM tempTable")); Schema output_schema = Schema.builder().addInt32Field("fout_int").addStringField("fout_string").build(); assertThat(result.getSchema(), equalTo(output_schema)); Row output = Row.withSchema(output_schema).addValues(1, "strstr").build(); PAssert.that(result).containsInAnyOrder(output); pipeline.run(); } On Tue, Jan 24, 2023 at 8:13 AM Talat Uyarer <tuya...@paloaltonetworks.com> wrote: > Hi Kenn, > > Thank you for replying back to my email. > > I was under the same impression about Calcite. But I wrote a test on > Calcite 1.28 too. It is working without issue that I see on BEAM > > Here is my test case. If you want you can also run on Calcite. Please put > under core/src/test/resources/sql as text file. and Run CoreQuidemTest > class. > > !use post > !set outputformat mysql > > #Test aliases with with clause > WITH tempTable(id, v) AS (select "hr"."emps"."empid" as id, > "hr"."emps"."name" as v from "hr"."emps") > SELECT tempTable.id as id, tempTable.v as "value" FROM tempTable WHERE > tempTable.v <> '11' ; > +-----+-----------+ > | ID | value | > +-----+-----------+ > | 100 | Bill | > | 110 | Theodore | > | 150 | Sebastian | > | 200 | Eric | > +-----+-----------+ > (4 rows) > > !ok > > > On Mon, Jan 23, 2023 at 10:16 AM Kenneth Knowles <k...@apache.org> wrote: > >> Looking at the code that turns a logical CalcRel into a BeamCalcRel I do >> not see any obvious cause for this: >> https://github.com/apache/beam/blob/b3aa2e89489898f8c760294ba4dba2310ac53e70/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java#L69 >> <https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_beam_blob_b3aa2e89489898f8c760294ba4dba2310ac53e70_sdks_java_extensions_sql_src_main_java_org_apache_beam_sdk_extensions_sql_impl_rule_BeamCalcRule.java-23L69&d=DwMFaQ&c=V9IgWpI5PvzTw83UyHGVSoW3Uc1MFWe5J8PTfkrzVSo&r=BkW1L6EF7ergAVYDXCo-3Vwkpy6qjsWAz7_GD7pAR8g&m=KXc2qSceL6qFbFnQ_2qUOHr9mKuc6zYY8rJTNZC8p_wTcNs4M6mHQoCuoc4JfeaA&s=KjzplEf29oFB6uivvdjixpQiArWtfV-1SXpALL-ugEM&e=> >> >> I don't like to guess that upstream libraries have the bug, but in this >> case I wonder if the alias is lost in the Calcite optimizer rule for >> merging the projects and filters into a Calc. >> >> Kenn >> >> On Mon, Jan 23, 2023 at 10:13 AM Kenneth Knowles <k...@apache.org> wrote: >> >>> I am not sure I understand the question, but I do see an issue. >>> >>> Context: "CalcRel" is an optimized relational operation that is somewhat >>> like ParDo, with a small snippet of a single-assignment DSL embedded in it. >>> Calcite will choose to merge all the projects and filters into the node, >>> and then generates Java bytecode to directly execute the DSL. >>> >>> Problem: it looks like the CalcRel has output columns with aliases "id" >>> and "v" where it should have output columns with aliases "id" and "value". >>> >>> Kenn >>> >>> On Thu, Jan 19, 2023 at 6:01 PM Ahmet Altay <al...@google.com> wrote: >>> >>>> Adding: @Andrew Pilloud <apill...@google.com> @Kenneth Knowles >>>> <k...@google.com> >>>> >>>> On Thu, Jan 12, 2023 at 12:31 PM Talat Uyarer via user < >>>> user@beam.apache.org> wrote: >>>> >>>>> Hi All, >>>>> >>>>> I am using Beam 2.43 with Calcite SQL with Java. >>>>> >>>>> I have a query with a WITH clause and some aliasing. Looks like Beam >>>>> Query optimizer after optimizing my query, it drops Select statement's >>>>> aliases. Can you help me to identify where the problem is ? >>>>> >>>>> This is my query >>>>> INFO: SQL: >>>>> WITH `tempTable` (`id`, `v`) AS (SELECT >>>>> `PCOLLECTION`.`f_nestedRow`.`f_nestedInt` AS `id`, >>>>> `PCOLLECTION`.`f_nestedRow`.`f_nestedString` AS `v` >>>>> FROM `beam`.`PCOLLECTION` AS `PCOLLECTION`) (SELECT `tempTable`.`id` >>>>> AS `id`, `tempTable`.`v` AS `value` >>>>> FROM `tempTable` AS `tempTable` >>>>> WHERE `tempTable`.`v` <> '11') >>>>> >>>>> This is Calcite Plan look at LogicalProject(id=[$0], value=[$1]) in >>>>> SQL plan. >>>>> >>>>> Jan 12, 2023 12:19:08 PM >>>>> org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner >>>>> convertToBeamRel >>>>> INFO: SQLPlan> >>>>> LogicalProject(id=[$0], value=[$1]) >>>>> LogicalFilter(condition=[<>($1, '11')]) >>>>> LogicalProject(id=[$1.f_nestedInt], v=[$1.f_nestedString]) >>>>> BeamIOSourceRel(table=[[beam, PCOLLECTION]]) >>>>> >>>>> But Beam Plan does not have a LogicalProject(id=[$0], value=[$1]) or >>>>> similar. >>>>> >>>>> Jan 12, 2023 12:19:08 PM >>>>> org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner >>>>> convertToBeamRel >>>>> INFO: BEAMPlan> >>>>> BeamCalcRel(expr#0..1=[{inputs}], expr#2=[$t1.f_nestedInt], >>>>> expr#3=[$t1.f_nestedString], expr#4=['11':VARCHAR], expr#5=[<>($t3, $t4)], >>>>> id=[$t2], v=[$t3], $condition=[$t5]) >>>>> BeamIOSourceRel(table=[[beam, PCOLLECTION]]) >>>>> >>>>> >>>>> Thanks >>>>> >>>>