This is an automated email from the ASF dual-hosted git repository.

anton pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 2e17f12  [BEAM-7526] Fix toBeamValue logic in BigQueryUtils
     new 01f14a0  Merge pull request #8814 from charithe/beam-7526
2e17f12 is described below

commit 2e17f12fa8fea43fb93a47f1770e163bac65cb3a
Author: Charith Ellawala <charith.ellaw...@gmail.com>
AuthorDate: Tue Jun 11 11:09:11 2019 +0100

    [BEAM-7526] Fix toBeamValue logic in BigQueryUtils
---
 .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java  | 7 +++++--
 .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java   | 9 +++++++--
 2 files changed, 12 insertions(+), 4 deletions(-)

diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
index 1a87875..3ef1507 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
@@ -444,12 +444,15 @@ public class BigQueryUtils {
     if (jsonBQValue instanceof List) {
       return ((List<Object>) jsonBQValue)
           .stream()
+              .map(v -> ((Map<String, Object>) v).get("v"))
               .map(v -> toBeamValue(fieldType.getCollectionElementType(), v))
               .collect(toList());
     }
 
-    if (jsonBQValue instanceof TableRow) {
-      return toBeamRow(fieldType.getRowSchema(), (TableRow) jsonBQValue);
+    if (jsonBQValue instanceof Map) {
+      TableRow tr = new TableRow();
+      tr.putAll((Map<String, Object>) jsonBQValue);
+      return toBeamRow(fieldType.getRowSchema(), tr);
     }
 
     throw new UnsupportedOperationException(
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
index 3315598..2226be9 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
@@ -131,7 +131,11 @@ public class BigQueryUtilsTest {
       Row.withSchema(ARRAY_TYPE).addValues((Object) Arrays.asList(123L, 
124L)).build();
 
   private static final TableRow BQ_ARRAY_ROW =
-      new TableRow().set("ids", Arrays.asList("123", "124"));
+      new TableRow()
+          .set(
+              "ids",
+              Arrays.asList(
+                  Collections.singletonMap("v", "123"), 
Collections.singletonMap("v", "124")));
 
   private static final Row ROW_ROW = 
Row.withSchema(ROW_TYPE).addValues(FLAT_ROW).build();
 
@@ -141,7 +145,8 @@ public class BigQueryUtilsTest {
       Row.withSchema(ARRAY_ROW_TYPE).addValues((Object) 
Arrays.asList(FLAT_ROW)).build();
 
   private static final TableRow BQ_ARRAY_ROW_ROW =
-      new TableRow().set("rows", Collections.singletonList(BQ_FLAT_ROW));
+      new TableRow()
+          .set("rows", Collections.singletonList(Collections.singletonMap("v", 
BQ_FLAT_ROW)));
 
   private static final TableSchema BQ_FLAT_TYPE =
       new TableSchema().setFields(Arrays.asList(ID, VALUE, NAME, TIMESTAMP, 
VALID));

Reply via email to