Hi Rui,
I was trying out a use case where we have a map with key as string and
value as Row. When we try to access the primitive field in the Row we are
getting below exception.
Caused by: java.lang.NoSuchFieldException: color
at java.lang.Class.getDeclaredField(Class.java:2070)
at
org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.runtime.SqlFunctions.structAccess(SqlFunctions.java:2405)
Below is the schema and the sql query used:
Schema primitiveFieldsScema =
Schema.builder().addStringField("color").build();
Schema inputSchema =
Schema.builder().addMapField("mapWithValueAsRow", FieldType.STRING,
FieldType.row(primitiveFieldsScema)).build();
Map<String, Row> mapWithValueAsRow = new HashMap<>();
Row row =
Row.withSchema(primitiveFieldsScema).addValue("RED").build();
mapWithValueAsRow.put("key", row);
Row rowOfMap =
Row.withSchema(inputSchema).addValue(mapWithValueAsRow).build();
Query used:
* select PCOLLECTION.mapWithValueAsRow['key'].color as color
from PCOLLECTION*
In git there are scenario
<https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMapTest.java>
of accessing map using select query. But in those values are all of
primitive types. In my case the value is of type Row.
So can you let me know whether this is supported or this is a bug ?
*Thanks & Regards,*
*Vishwas *
On Thu, Jun 20, 2019 at 11:25 PM Rui Wang <[email protected]> wrote:
> Oops I made a mistake, I didn't work on[1] but actually [2]
>
> [1]:
> https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslNestedRowsTest.java
> [2]:
> https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
>
> -Rui
>
> On Thu, Jun 20, 2019 at 9:58 AM Rui Wang <[email protected]> wrote:
>
>> I wrote some tests on nested row selection in BeamSQL[1]. Those test
>> cases test some behaviors of nested row selection that BeamSQL supports(but
>> it's not a complete list).
>>
>> You could check what are tested so that are supported. Also it's welcome
>> to extend those tests to cover more behaviors.
>>
>>
>> [1]:
>> https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslNestedRowsTest.java
>>
>>
>> -Rui
>>
>> On Thu, Jun 20, 2019 at 4:25 AM Andrew Pilloud <[email protected]>
>> wrote:
>>
>>> Hi Alex,
>>>
>>> Unfortunately you are receiving an UnsupportedOperationException because
>>> selecting nested rows is not supported by Calcite. You select fields out of
>>> the nested row but not a row itself. There are some recent bug fixes in
>>> this area in Calcite 1.20, so it might be worth trying that. There has been
>>> a lot of work on this area in Calcite in the past year so it is also
>>> possible the work that remains is in Beam. We have a bug open on the issue
>>> in Beam: https://issues.apache.org/jira/browse/BEAM-5189
>>>
>>> Beam is repackaging Calcite by copying and relocating the code using
>>> the gradle relocate rule. The code is unchanged, but
>>> 'org.apache.beam.repackaged.beam_sdks_java_extensions_sql' is added to the
>>> class path so it won't conflict with Calcite in use by some of the Beam
>>> runners. If you want to change the version to a local snapshot of calcite
>>> for development you can modify it here:
>>> https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/build.gradle#L72
>>>
>>> Andrew
>>>
>>> On Thu, Jun 20, 2019 at 3:02 AM alex goos <[email protected]> wrote:
>>>
>>>> Beam 2.13, I'm trying to do SqlTransform on Row records with
>>>> hierarchical structures. Records having (nullable) arrays of (nullabke)
>>>> sub-records:
>>>>
>>>> "select p.recordType, p.listOfServiceData.seqOf[0].ratingGroup as
>>>> ratingGroup, p.abcdAddress.seqOf[0] as ABCD_IP FROM PCOLLECTION "
>>>>
>>>> The code resposible for the Error when accessing subsctructures seems
>>>> to come from a repackaged Calcite Library: Where do I start asking
>>>> around/fixing the issue ?! Here? at the Apache Calcite mailing list? How is
>>>> Beam "repackageing" Calcite-core ? Manually,by importing code ?!
>>>>
>>>> Exception in thread "main" java.lang.UnsupportedOperationException:
>>>> class
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.rex.RexFieldAccess:
>>>> ITEM($28.seqOf, 0).ratingGroup
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.util.Util.needToImplement(Util.java:955)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.sql2rel.RelStructuredTypeFlattener.flattenProjection(RelStructuredTypeFlattener.java:662)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.sql2rel.RelStructuredTypeFlattener.flattenProjections(RelStructuredTypeFlattener.java:587)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.sql2rel.RelStructuredTypeFlattener.rewriteRel(RelStructuredTypeFlattener.java:501)
>>>> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>> at
>>>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>> at
>>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>> at java.lang.reflect.Method.invoke(Method.java:498)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.util.ReflectUtil.invokeVisitorInternal(ReflectUtil.java:257)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.util.ReflectUtil.invokeVisitor(ReflectUtil.java:214)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.util.ReflectUtil$1.invokeVisitor(ReflectUtil.java:464)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.sql2rel.RelStructuredTypeFlattener$RewriteRelVisitor.visit(RelStructuredTypeFlattener.java:768)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.sql2rel.RelStructuredTypeFlattener.rewrite(RelStructuredTypeFlattener.java:195)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.sql2rel.SqlToRelConverter.flattenTypes(SqlToRelConverter.java:468)
>>>> at
>>>> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.prepare.PlannerImpl.rel(PlannerImpl.java:236)
>>>> at
>>>> org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner.convertToBeamRel(CalciteQueryPlanner.java:129)
>>>> at
>>>> org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv.parseQuery(BeamSqlEnv.java:87)
>>>> at
>>>> org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:122)
>>>> at
>>>> org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:82)
>>>> at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
>>>> at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:471)
>>>>
>>>> Many thanks!
>>>>
>>>