You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/03/03 19:33:38 UTC

[GitHub] [beam] TheNeuralBit commented on a change in pull request #16988: BEAM-14026 - Fixes bug related to Unnesting nested rows in an array

TheNeuralBit commented on a change in pull request #16988:
URL: https://github.com/apache/beam/pull/16988#discussion_r818990956



##########
File path: sdks/java/extensions/sql/build.gradle
##########
@@ -206,6 +206,14 @@ task runBasicExample(type: JavaExec) {
   args = ["--runner=DirectRunner"]
 }
 
+// Run basic SQL example
+task runNestedRowInArrayExample(type: JavaExec) {
+  description = "Run basic SQL example"
+  mainClass = "org.apache.beam.sdk.extensions.sql.example.BeamSqlUnnestExample"
+  classpath = sourceSets.main.runtimeClasspath
+  args = ["--runner=DirectRunner"]
+}

Review comment:
       Do we need this example? It looks like the same behavior is verified continuously in [BeamSqlDslUnnestRowsTest.java](https://github.com/apache/beam/pull/16988/files#diff-8c0c041856075a2f10a667e185965b258fb8dd695a9a7ab9e4fdb417499fd424)

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
##########
@@ -108,11 +109,32 @@
   /** Return the list of data values. */
   public abstract List<Object> getValues();
 
+  /** This is recursive call to get all the values of the nested rows.
+  The recusion is bounded by the amount of nesting with in the data
+   This mirrors the unnest behavior of calcite towards schema **/
+  public List<Object> getNestedRowBaseValues() {
+    return IntStream.range(0, getFieldCount())
+            .mapToObj(i -> {
+              List<Object> values = new ArrayList<>();
+              FieldType fieldType = this.getSchema().getField(i).getType();
+              if(fieldType.getTypeName().equals(TypeName.ROW)) {
+                Row row = this.getBaseValue(i, Row.class);
+                List<Object> rowValues = row.getNestedRowBaseValues();
+                if(null != rowValues) {
+                  values.addAll(rowValues);
+                }
+              } else {
+                values.add(this.getBaseValue(i));
+              }
+              return values.stream();
+            }).flatMap(Function.identity()).collect(Collectors.toList());
+  }

Review comment:
       I'd prefer not to add this logic to Row's public API. It's specific to the needs in unnest. Would it be possible to just define it as a helper there?

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
##########
@@ -537,7 +559,7 @@ static int deepHashCodeForMap(
     }
 
     static boolean deepEqualsForCollection(
-        Collection<Object> a, Collection<Object> b, Schema.FieldType elementType) {
+            Collection<Object> a, Collection<Object> b, Schema.FieldType elementType) {

Review comment:
       I think unnecessary whitespace changes like this are what's making our CI unhappy. Can you run spotless to format the code? `./gradlew spotlessApply`
   
   You might also double check there aren't other checkstyle failures: `./gradlew :sdks:java:extensions:sql:checkstyleMain :sdks:java:extensions:sql:checkstyleTest`




-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org