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 <ag...@mac.com> 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! >