[ 
https://issues.apache.org/jira/browse/BEAM-7623?focusedWorklogId=284444&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-284444
 ]

ASF GitHub Bot logged work on BEAM-7623:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 29/Jul/19 17:46
            Start Date: 29/Jul/19 17:46
    Worklog Time Spent: 10m 
      Work Description: akedin commented on pull request #9181: [BEAM-7623] Add 
support to select MAP with Row as values in Beam SQL
URL: https://github.com/apache/beam/pull/9181#discussion_r308354590
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
 ##########
 @@ -430,4 +432,81 @@ public void testNullDatetimeFields() {
 
     pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
   }
+
+  @Test
+  public void testMapWithRowAsValue() {
+
+    Schema inputSchema =
+        Schema.builder()
+            .addMapField("mapWithValueAsRow", FieldType.STRING, 
FieldType.row(rowWithArraySchema))
+            .build();
+
+    Map<String, Row> mapWithValueAsRow = new HashMap<>();
+    Row complexRow =
+        Row.withSchema(rowWithArraySchema)
+            .addValues("RED", 5L, Arrays.asList(10L, 20L, 30L))
+            .build();
+    mapWithValueAsRow.put("key", complexRow);
+
+    Row rowOfMap = 
Row.withSchema(inputSchema).addValue(mapWithValueAsRow).build();
+
+    PCollection<Row> outputRow =
+        pipeline
+            .apply(Create.of(rowOfMap))
+            .setRowSchema(inputSchema)
+            .apply(
+                SqlTransform.query(
+                    "select  PCOLLECTION.mapWithValueAsRow['key'].field1 as 
color, PCOLLECTION.mapWithValueAsRow['key'].field3[2]  as num   from 
PCOLLECTION"));
+
+    Row expectedRow =
+        
Row.withSchema(Schema.builder().addStringField("color").addInt64Field("num").build())
+            .addValues("RED", 20L)
+            .build();
+
+    PAssert.that(outputRow).containsInAnyOrder(expectedRow);
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(1));
+  }
+
+  @Test
+  public void testMapWithNullRowFields() {
+
+    Schema nullableInnerSchema =
+        Schema.builder()
+            .addNullableField("strField", FieldType.STRING)
+            .addNullableField("arrField", FieldType.array(FieldType.INT64))
+            .build();
+    Schema inputSchema =
+        Schema.builder()
+            .addMapField("mapField", FieldType.STRING, 
FieldType.row(nullableInnerSchema))
+            .addNullableField(
+                "nullableMapField",
+                FieldType.map(FieldType.STRING, 
FieldType.row(nullableInnerSchema)))
+            .build();
+
+    Row mapValue = Row.withSchema(nullableInnerSchema).addValues("str", 
null).build();
+    Map<String, Row> mapWithValueAsRow = new HashMap<>();
+    mapWithValueAsRow.put("key", mapValue);
+
+    Row inputRow = Row.withSchema(inputSchema).addValues(mapWithValueAsRow, 
null).build();
+
+    PCollection<Row> outputRow =
+        pipeline
+            .apply(Create.of(inputRow))
+            .setRowSchema(inputSchema)
+            .apply(
+                SqlTransform.query(
+                    "select PCOLLECTION.mapField['key'].strField as str, 
PCOLLECTION.mapField['key'].arrField[1] as arr, 
PCOLLECTION.nullableMapField['key'].arrField[1] as nullableField  from 
PCOLLECTION"));
 
 Review comment:
   I think the queries need comments at this level of complexity, it's hard to 
decipher what is returned in which sub-expression.
   
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 284444)
    Time Spent: 0.5h  (was: 20m)

> Support select MAP with Row as values
> -------------------------------------
>
>                 Key: BEAM-7623
>                 URL: https://issues.apache.org/jira/browse/BEAM-7623
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Rui Wang
>            Priority: Major
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> {code}
>      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
> {code}
> With exception:
> {code}
> java.lang.RuntimeException: CalcFn failed to evaluate: {
>   final java.util.Map inp0_ = ((org.apache.beam.sdk.values.Row) 
> c.element()).getMap(0);
>   
> c.output(org.apache.beam.sdk.values.Row.withSchema(outputSchema).addValue(org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.mapItemOptional(inp0_,
>  "key") == null ? (String) null : (String) 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.structAccess(org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.mapItemOptional(inp0_,
>  "key"), 0, "color")).build());
> }
> org.apache.beam.sdk.Pipeline$PipelineExecutionException: 
> java.lang.RuntimeException: CalcFn failed to evaluate: {
>   final java.util.Map inp0_ = ((org.apache.beam.sdk.values.Row) 
> c.element()).getMap(0);
>   
> c.output(org.apache.beam.sdk.values.Row.withSchema(outputSchema).addValue(org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.mapItemOptional(inp0_,
>  "key") == null ? (String) null : (String) 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.structAccess(org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.mapItemOptional(inp0_,
>  "key"), 0, "color")).build());
> }
>       at 
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:348)
>       at 
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:318)
>       at 
> org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:213)
>       at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
>       at org.apache.beam.sdk.Pipeline.run(Pipeline.java:313)
>       at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:350)
>       at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:331)
>       at 
> org.apache.beam.sdk.extensions.sql.BeamSqlMapTest.testAccessMapElementWithRowValue(BeamSqlMapTest.java:155)
>       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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:265)
>       at 
> org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:319)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>       at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:349)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>       at org.junit.runners.ParentRunner$3.run(ParentRunner.java:314)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:312)
>       at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:292)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:396)
>       at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
>       at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
>       at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
>       at 
> org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
>       at 
> org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
>       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.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
>       at 
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>       at 
> org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
>       at 
> org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
>       at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>       at 
> org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
>       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.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
>       at 
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>       at 
> org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
>       at 
> org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
>       at 
> org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
>       at 
> org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
>       at 
> org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at 
> org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.RuntimeException: CalcFn failed to evaluate: {
>   final java.util.Map inp0_ = ((org.apache.beam.sdk.values.Row) 
> c.element()).getMap(0);
>   
> c.output(org.apache.beam.sdk.values.Row.withSchema(outputSchema).addValue(org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.mapItemOptional(inp0_,
>  "key") == null ? (String) null : (String) 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.structAccess(org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.mapItemOptional(inp0_,
>  "key"), 0, "color")).build());
> }
>       at 
> org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel$CalcFn.processElement(BeamCalcRel.java:253)
> Caused by: 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.CalciteException: 
> Failed to access field 'color' of object of type 
> org.apache.beam.sdk.values.RowWithStorage
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>       at 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.Resources$ExInstWithCause.ex(Resources.java:463)
>       at 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.structAccess(SqlFunctions.java:2408)
>       at SC.eval0(Unknown Source)
>       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.sdks.java.extensions.sql.repackaged.org.codehaus.janino.ScriptEvaluator.evaluate(ScriptEvaluator.java:798)
>       at 
> org.apache.beam.sdks.java.extensions.sql.repackaged.org.codehaus.janino.ScriptEvaluator.evaluate(ScriptEvaluator.java:790)
>       at 
> org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel$CalcFn.processElement(BeamCalcRel.java:250)
>       at 
> org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel$CalcFn$DoFnInvoker.invokeProcessElement(Unknown
>  Source)
>       at 
> org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.invokeProcessElement(SimpleDoFnRunner.java:214)
>       at 
> org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.processElement(SimpleDoFnRunner.java:179)
>       at 
> org.apache.beam.repackaged.direct_java.runners.core.SimplePushbackSideInputDoFnRunner.processElementInReadyWindows(SimplePushbackSideInputDoFnRunner.java:78)
>       at 
> org.apache.beam.runners.direct.ParDoEvaluator.processElement(ParDoEvaluator.java:211)
>       at 
> org.apache.beam.runners.direct.DoFnLifecycleManagerRemovingTransformEvaluator.processElement(DoFnLifecycleManagerRemovingTransformEvaluator.java:54)
>       at 
> org.apache.beam.runners.direct.DirectTransformExecutor.processElements(DirectTransformExecutor.java:160)
>       at 
> org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:124)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.NoSuchFieldException: color
>       at java.lang.Class.getDeclaredField(Class.java:2070)
>       at 
> org.apache.beam.repackaged.sql.org.apache.calcite.runtime.SqlFunctions.structAccess(SqlFunctions.java:2405)
>       ... 21 more
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to