Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-24 Thread Rui Wang
That is great. Feel free to send the patch and I can review it.

-Rui

On Mon, Feb 24, 2020, 3:54 PM Talat Uyarer 
wrote:

> Hi Rui,
>
> I solved the issue. After 1.21 version they are not getting flattened in
> LogicalPlan.
>
> Thanks for your help. I am going to create a patch for it.
>
> Talat
>
> On Sat, Feb 15, 2020 at 6:26 PM Rui Wang  wrote:
>
>> Because Calcite flattened Row so BeamSQL didn't need to deal with nested
>> Row structure (as they were flattened in LogicalPlan).
>>
>> Depends on how that patch works. Nested row might not immediately work
>> after you apply that patch.
>>
>>
>> -Rui
>>
>> On Fri, Feb 14, 2020 at 3:14 PM Talat Uyarer <
>> tuya...@paloaltonetworks.com> wrote:
>>
>>> Do you mean they were flattened before by calcite or Does beam flatten
>>> them too ?
>>>
>>>
>>>
>>> On Fri, Feb 14, 2020 at 1:21 PM Rui Wang  wrote:
>>>
 Nested row types might be less well supported (r.g. Row) because they
 were flattened before anyway.


 -Rui

 On Fri, Feb 14, 2020 at 12:14 PM Talat Uyarer <
 tuya...@paloaltonetworks.com> wrote:

> Thank you for your response.
> I saw it and applied patch on calcite 1.20. However I realized
> BeamCalRel does not generate right code [1]to turn back Beam types. I am
> working on that now. Please let me know if apache beam support nested row
> types but I miss it.
>
>
> [1]
> https://github.com/apache/beam/blob/646f596988be9d6a739090f48d2fed07c8dfc17c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L167
> 
>
> On Fri, Feb 14, 2020 at 10:33 AM Rui Wang  wrote:
>
>> Calcite has improved to reconstruct ROW back in the output. See [1].
>> Beam need to update Calcite dependency to > 1.21 to adopt that.
>>
>>
>>
>> [1]: https://jira.apache.org/jira/browse/CALCITE-3138
>> 
>>
>>
>> -Rui
>>
>> On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer <
>> tuya...@paloaltonetworks.com> wrote:
>>
>>> Hi,
>>>
>>> I am trying to Beam SQL. But something is wrong. I have nested row
>>> records. I read them as Pcollection and apply Select * query and
>>> compare with initial rows. Looks like nested rows are flatten by 
>>> calcite.
>>> How do you have any idea how can I avoid this?
>>>
>>> I added a same testcase for my issue:
>>>
>>> Schema nestedSchema =
>>> Schema.builder()
>>> .addInt32Field("f_nestedInt")
>>> .addStringField("f_nestedString")
>>> .addInt32Field("f_nestedIntPlusOne")
>>> .build();
>>> Schema inputType =
>>> Schema.builder().addInt32Field("f_int").addRowField("f_row", 
>>> nestedSchema).build();
>>>
>>> PCollection input =
>>> pipeline.apply(
>>> Create.of(
>>> Row.withSchema(inputType)
>>> .addValues(
>>> 1, Row.withSchema(nestedSchema).addValues(312, 
>>> "CC", 313).build())
>>> .build())
>>> .withRowSchema(inputType))
>>> .setRowSchema(inputType);
>>>
>>> PCollection result =
>>> input
>>> .apply(
>>> SqlTransform.query(
>>> "SELECT * FROM PCOLLECTION"));
>>>
>>> PAssert.that(result)
>>> .containsInAnyOrder(Row.withSchema(inputType)
>>> .addValues(
>>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>>> 313).build())
>>> .build());
>>>
>>>
>>> Thank you so much in advance.
>>>
>>>


Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-24 Thread Talat Uyarer
Hi Rui,

I solved the issue. After 1.21 version they are not getting flattened in
LogicalPlan.

Thanks for your help. I am going to create a patch for it.

Talat

On Sat, Feb 15, 2020 at 6:26 PM Rui Wang  wrote:

> Because Calcite flattened Row so BeamSQL didn't need to deal with nested
> Row structure (as they were flattened in LogicalPlan).
>
> Depends on how that patch works. Nested row might not immediately work
> after you apply that patch.
>
>
> -Rui
>
> On Fri, Feb 14, 2020 at 3:14 PM Talat Uyarer 
> wrote:
>
>> Do you mean they were flattened before by calcite or Does beam flatten
>> them too ?
>>
>>
>>
>> On Fri, Feb 14, 2020 at 1:21 PM Rui Wang  wrote:
>>
>>> Nested row types might be less well supported (r.g. Row) because they
>>> were flattened before anyway.
>>>
>>>
>>> -Rui
>>>
>>> On Fri, Feb 14, 2020 at 12:14 PM Talat Uyarer <
>>> tuya...@paloaltonetworks.com> wrote:
>>>
 Thank you for your response.
 I saw it and applied patch on calcite 1.20. However I realized
 BeamCalRel does not generate right code [1]to turn back Beam types. I am
 working on that now. Please let me know if apache beam support nested row
 types but I miss it.


 [1]
 https://github.com/apache/beam/blob/646f596988be9d6a739090f48d2fed07c8dfc17c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L167
 

 On Fri, Feb 14, 2020 at 10:33 AM Rui Wang  wrote:

> Calcite has improved to reconstruct ROW back in the output. See [1].
> Beam need to update Calcite dependency to > 1.21 to adopt that.
>
>
>
> [1]: https://jira.apache.org/jira/browse/CALCITE-3138
> 
>
>
> -Rui
>
> On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer <
> tuya...@paloaltonetworks.com> wrote:
>
>> Hi,
>>
>> I am trying to Beam SQL. But something is wrong. I have nested row
>> records. I read them as Pcollection and apply Select * query and
>> compare with initial rows. Looks like nested rows are flatten by calcite.
>> How do you have any idea how can I avoid this?
>>
>> I added a same testcase for my issue:
>>
>> Schema nestedSchema =
>> Schema.builder()
>> .addInt32Field("f_nestedInt")
>> .addStringField("f_nestedString")
>> .addInt32Field("f_nestedIntPlusOne")
>> .build();
>> Schema inputType =
>> Schema.builder().addInt32Field("f_int").addRowField("f_row", 
>> nestedSchema).build();
>>
>> PCollection input =
>> pipeline.apply(
>> Create.of(
>> Row.withSchema(inputType)
>> .addValues(
>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>> 313).build())
>> .build())
>> .withRowSchema(inputType))
>> .setRowSchema(inputType);
>>
>> PCollection result =
>> input
>> .apply(
>> SqlTransform.query(
>> "SELECT * FROM PCOLLECTION"));
>>
>> PAssert.that(result)
>> .containsInAnyOrder(Row.withSchema(inputType)
>> .addValues(
>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>> 313).build())
>> .build());
>>
>>
>> Thank you so much in advance.
>>
>>


Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-15 Thread Rui Wang
Because Calcite flattened Row so BeamSQL didn't need to deal with nested
Row structure (as they were flattened in LogicalPlan).

Depends on how that patch works. Nested row might not immediately work
after you apply that patch.


-Rui

On Fri, Feb 14, 2020 at 3:14 PM Talat Uyarer 
wrote:

> Do you mean they were flattened before by calcite or Does beam flatten
> them too ?
>
>
>
> On Fri, Feb 14, 2020 at 1:21 PM Rui Wang  wrote:
>
>> Nested row types might be less well supported (r.g. Row) because they
>> were flattened before anyway.
>>
>>
>> -Rui
>>
>> On Fri, Feb 14, 2020 at 12:14 PM Talat Uyarer <
>> tuya...@paloaltonetworks.com> wrote:
>>
>>> Thank you for your response.
>>> I saw it and applied patch on calcite 1.20. However I realized
>>> BeamCalRel does not generate right code [1]to turn back Beam types. I am
>>> working on that now. Please let me know if apache beam support nested row
>>> types but I miss it.
>>>
>>>
>>> [1]
>>> https://github.com/apache/beam/blob/646f596988be9d6a739090f48d2fed07c8dfc17c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L167
>>> 
>>>
>>> On Fri, Feb 14, 2020 at 10:33 AM Rui Wang  wrote:
>>>
 Calcite has improved to reconstruct ROW back in the output. See [1].
 Beam need to update Calcite dependency to > 1.21 to adopt that.



 [1]: https://jira.apache.org/jira/browse/CALCITE-3138
 


 -Rui

 On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer <
 tuya...@paloaltonetworks.com> wrote:

> Hi,
>
> I am trying to Beam SQL. But something is wrong. I have nested row
> records. I read them as Pcollection and apply Select * query and
> compare with initial rows. Looks like nested rows are flatten by calcite.
> How do you have any idea how can I avoid this?
>
> I added a same testcase for my issue:
>
> Schema nestedSchema =
> Schema.builder()
> .addInt32Field("f_nestedInt")
> .addStringField("f_nestedString")
> .addInt32Field("f_nestedIntPlusOne")
> .build();
> Schema inputType =
> Schema.builder().addInt32Field("f_int").addRowField("f_row", 
> nestedSchema).build();
>
> PCollection input =
> pipeline.apply(
> Create.of(
> Row.withSchema(inputType)
> .addValues(
> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
> 313).build())
> .build())
> .withRowSchema(inputType))
> .setRowSchema(inputType);
>
> PCollection result =
> input
> .apply(
> SqlTransform.query(
> "SELECT * FROM PCOLLECTION"));
>
> PAssert.that(result)
> .containsInAnyOrder(Row.withSchema(inputType)
> .addValues(
> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
> 313).build())
> .build());
>
>
> Thank you so much in advance.
>
>


Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-14 Thread Talat Uyarer
Do you mean they were flattened before by calcite or Does beam flatten them
too ?



On Fri, Feb 14, 2020 at 1:21 PM Rui Wang  wrote:

> Nested row types might be less well supported (r.g. Row) because they were
> flattened before anyway.
>
>
> -Rui
>
> On Fri, Feb 14, 2020 at 12:14 PM Talat Uyarer <
> tuya...@paloaltonetworks.com> wrote:
>
>> Thank you for your response.
>> I saw it and applied patch on calcite 1.20. However I realized BeamCalRel
>> does not generate right code [1]to turn back Beam types. I am working on
>> that now. Please let me know if apache beam support nested row types but I
>> miss it.
>>
>>
>> [1]
>> https://github.com/apache/beam/blob/646f596988be9d6a739090f48d2fed07c8dfc17c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L167
>> 
>>
>> On Fri, Feb 14, 2020 at 10:33 AM Rui Wang  wrote:
>>
>>> Calcite has improved to reconstruct ROW back in the output. See [1].
>>> Beam need to update Calcite dependency to > 1.21 to adopt that.
>>>
>>>
>>>
>>> [1]: https://jira.apache.org/jira/browse/CALCITE-3138
>>> 
>>>
>>>
>>> -Rui
>>>
>>> On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer <
>>> tuya...@paloaltonetworks.com> wrote:
>>>
 Hi,

 I am trying to Beam SQL. But something is wrong. I have nested row
 records. I read them as Pcollection and apply Select * query and
 compare with initial rows. Looks like nested rows are flatten by calcite.
 How do you have any idea how can I avoid this?

 I added a same testcase for my issue:

 Schema nestedSchema =
 Schema.builder()
 .addInt32Field("f_nestedInt")
 .addStringField("f_nestedString")
 .addInt32Field("f_nestedIntPlusOne")
 .build();
 Schema inputType =
 Schema.builder().addInt32Field("f_int").addRowField("f_row", 
 nestedSchema).build();

 PCollection input =
 pipeline.apply(
 Create.of(
 Row.withSchema(inputType)
 .addValues(
 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
 313).build())
 .build())
 .withRowSchema(inputType))
 .setRowSchema(inputType);

 PCollection result =
 input
 .apply(
 SqlTransform.query(
 "SELECT * FROM PCOLLECTION"));

 PAssert.that(result)
 .containsInAnyOrder(Row.withSchema(inputType)
 .addValues(
 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
 313).build())
 .build());


 Thank you so much in advance.




Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-14 Thread Rui Wang
Nested row types might be less well supported (r.g. Row) because they were
flattened before anyway.


-Rui

On Fri, Feb 14, 2020 at 12:14 PM Talat Uyarer 
wrote:

> Thank you for your response.
> I saw it and applied patch on calcite 1.20. However I realized BeamCalRel
> does not generate right code [1]to turn back Beam types. I am working on
> that now. Please let me know if apache beam support nested row types but I
> miss it.
>
>
> [1]
> https://github.com/apache/beam/blob/646f596988be9d6a739090f48d2fed07c8dfc17c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L167
>
> On Fri, Feb 14, 2020 at 10:33 AM Rui Wang  wrote:
>
>> Calcite has improved to reconstruct ROW back in the output. See [1]. Beam
>> need to update Calcite dependency to > 1.21 to adopt that.
>>
>>
>>
>> [1]: https://jira.apache.org/jira/browse/CALCITE-3138
>> 
>>
>>
>> -Rui
>>
>> On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer <
>> tuya...@paloaltonetworks.com> wrote:
>>
>>> Hi,
>>>
>>> I am trying to Beam SQL. But something is wrong. I have nested row
>>> records. I read them as Pcollection and apply Select * query and
>>> compare with initial rows. Looks like nested rows are flatten by calcite.
>>> How do you have any idea how can I avoid this?
>>>
>>> I added a same testcase for my issue:
>>>
>>> Schema nestedSchema =
>>> Schema.builder()
>>> .addInt32Field("f_nestedInt")
>>> .addStringField("f_nestedString")
>>> .addInt32Field("f_nestedIntPlusOne")
>>> .build();
>>> Schema inputType =
>>> Schema.builder().addInt32Field("f_int").addRowField("f_row", 
>>> nestedSchema).build();
>>>
>>> PCollection input =
>>> pipeline.apply(
>>> Create.of(
>>> Row.withSchema(inputType)
>>> .addValues(
>>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>>> 313).build())
>>> .build())
>>> .withRowSchema(inputType))
>>> .setRowSchema(inputType);
>>>
>>> PCollection result =
>>> input
>>> .apply(
>>> SqlTransform.query(
>>> "SELECT * FROM PCOLLECTION"));
>>>
>>> PAssert.that(result)
>>> .containsInAnyOrder(Row.withSchema(inputType)
>>> .addValues(
>>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>>> 313).build())
>>> .build());
>>>
>>>
>>> Thank you so much in advance.
>>>
>>>


Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-14 Thread Talat Uyarer
Thank you for your response.
I saw it and applied patch on calcite 1.20. However I realized BeamCalRel
does not generate right code [1]to turn back Beam types. I am working on
that now. Please let me know if apache beam support nested row types but I
miss it.


[1]
https://github.com/apache/beam/blob/646f596988be9d6a739090f48d2fed07c8dfc17c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java#L167

On Fri, Feb 14, 2020 at 10:33 AM Rui Wang  wrote:

> Calcite has improved to reconstruct ROW back in the output. See [1]. Beam
> need to update Calcite dependency to > 1.21 to adopt that.
>
>
>
> [1]: https://jira.apache.org/jira/browse/CALCITE-3138
> 
>
>
> -Rui
>
> On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer 
> wrote:
>
>> Hi,
>>
>> I am trying to Beam SQL. But something is wrong. I have nested row
>> records. I read them as Pcollection and apply Select * query and
>> compare with initial rows. Looks like nested rows are flatten by calcite.
>> How do you have any idea how can I avoid this?
>>
>> I added a same testcase for my issue:
>>
>> Schema nestedSchema =
>> Schema.builder()
>> .addInt32Field("f_nestedInt")
>> .addStringField("f_nestedString")
>> .addInt32Field("f_nestedIntPlusOne")
>> .build();
>> Schema inputType =
>> Schema.builder().addInt32Field("f_int").addRowField("f_row", 
>> nestedSchema).build();
>>
>> PCollection input =
>> pipeline.apply(
>> Create.of(
>> Row.withSchema(inputType)
>> .addValues(
>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>> 313).build())
>> .build())
>> .withRowSchema(inputType))
>> .setRowSchema(inputType);
>>
>> PCollection result =
>> input
>> .apply(
>> SqlTransform.query(
>> "SELECT * FROM PCOLLECTION"));
>>
>> PAssert.that(result)
>> .containsInAnyOrder(Row.withSchema(inputType)
>> .addValues(
>> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
>> 313).build())
>> .build());
>>
>>
>> Thank you so much in advance.
>>
>>


Re: Beam SQL Nested Rows are flatten by Calcite

2020-02-14 Thread Rui Wang
Calcite has improved to reconstruct ROW back in the output. See [1]. Beam
need to update Calcite dependency to > 1.21 to adopt that.



[1]: https://jira.apache.org/jira/browse/CALCITE-3138


-Rui

On Thu, Feb 13, 2020 at 9:05 PM Talat Uyarer 
wrote:

> Hi,
>
> I am trying to Beam SQL. But something is wrong. I have nested row
> records. I read them as Pcollection and apply Select * query and
> compare with initial rows. Looks like nested rows are flatten by calcite.
> How do you have any idea how can I avoid this?
>
> I added a same testcase for my issue:
>
> Schema nestedSchema =
> Schema.builder()
> .addInt32Field("f_nestedInt")
> .addStringField("f_nestedString")
> .addInt32Field("f_nestedIntPlusOne")
> .build();
> Schema inputType =
> Schema.builder().addInt32Field("f_int").addRowField("f_row", 
> nestedSchema).build();
>
> PCollection input =
> pipeline.apply(
> Create.of(
> Row.withSchema(inputType)
> .addValues(
> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 
> 313).build())
> .build())
> .withRowSchema(inputType))
> .setRowSchema(inputType);
>
> PCollection result =
> input
> .apply(
> SqlTransform.query(
> "SELECT * FROM PCOLLECTION"));
>
> PAssert.that(result)
> .containsInAnyOrder(Row.withSchema(inputType)
> .addValues(
> 1, Row.withSchema(nestedSchema).addValues(312, "CC", 313).build())
> .build());
>
>
> Thank you so much in advance.
>
>


Beam SQL Nested Rows are flatten by Calcite

2020-02-13 Thread Talat Uyarer
Hi,

I am trying to Beam SQL. But something is wrong. I have nested row records.
I read them as Pcollection and apply Select * query and compare with
initial rows. Looks like nested rows are flatten by calcite. How do you
have any idea how can I avoid this?

I added a same testcase for my issue:

Schema nestedSchema =
Schema.builder()
.addInt32Field("f_nestedInt")
.addStringField("f_nestedString")
.addInt32Field("f_nestedIntPlusOne")
.build();
Schema inputType =
Schema.builder().addInt32Field("f_int").addRowField("f_row",
nestedSchema).build();

PCollection input =
pipeline.apply(
Create.of(
Row.withSchema(inputType)
.addValues(
1, Row.withSchema(nestedSchema).addValues(312,
"CC", 313).build())
.build())
.withRowSchema(inputType))
.setRowSchema(inputType);

PCollection result =
input
.apply(
SqlTransform.query(
"SELECT * FROM PCOLLECTION"));

PAssert.that(result)
.containsInAnyOrder(Row.withSchema(inputType)
.addValues(
1, Row.withSchema(nestedSchema).addValues(312, "CC", 313).build())
.build());


Thank you so much in advance.