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

bhulette pushed a commit to branch release-2.37.0
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/release-2.37.0 by this push:
     new 8487557  [release-2.37.0][BEAM-13959] Fix StorageApi writes when a 
field is named f (#16886)
8487557 is described below

commit 8487557de9f3b871d7f3948c09953c99a14c1005
Author: Brian Hulette <[email protected]>
AuthorDate: Fri Feb 18 08:27:28 2022 -0800

    [release-2.37.0][BEAM-13959] Fix StorageApi writes when a field is named f 
(#16886)
    
    Co-authored-by: reuvenlax <[email protected]>
---
 .../io/gcp/bigquery/TableRowToStorageApiProto.java |  46 +++-
 .../bigquery/TableRowToStorageApiProtoTest.java    | 294 +++++++++++++++++++--
 2 files changed, 313 insertions(+), 27 deletions(-)

diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java
index 63eb8a8..e74f1a2 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.io.gcp.bigquery;
 
 import static java.util.stream.Collectors.toList;
 
+import com.google.api.services.bigquery.model.TableCell;
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
@@ -87,13 +88,10 @@ public class TableRowToStorageApiProto {
     return Iterables.getOnlyElement(fileDescriptor.getMessageTypes());
   }
 
-  /**
-   * Given a BigQuery TableRow, returns a protocol-buffer message that can be 
used to write data
-   * using the BigQuery Storage API.
-   */
-  public static DynamicMessage messageFromTableRow(Descriptor descriptor, 
TableRow tableRow) {
+  public static DynamicMessage messageFromMap(
+      Descriptor descriptor, AbstractMap<String, Object> map) {
     DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor);
-    for (Map.Entry<String, Object> entry : tableRow.entrySet()) {
+    for (Map.Entry<String, Object> entry : map.entrySet()) {
       @Nullable
       FieldDescriptor fieldDescriptor = 
descriptor.findFieldByName(entry.getKey().toLowerCase());
       if (fieldDescriptor == null) {
@@ -108,6 +106,32 @@ public class TableRowToStorageApiProto {
     return builder.build();
   }
 
+  /**
+   * Given a BigQuery TableRow, returns a protocol-buffer message that can be 
used to write data
+   * using the BigQuery Storage API.
+   */
+  public static DynamicMessage messageFromTableRow(Descriptor descriptor, 
TableRow tableRow) {
+    @Nullable List<TableCell> cells = tableRow.getF();
+    if (cells != null) {
+      DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor);
+      if (cells.size() > descriptor.getFields().size()) {
+        throw new RuntimeException("TableRow contained too many fields");
+      }
+      for (int i = 0; i < cells.size(); ++i) {
+        TableCell cell = cells.get(i);
+        FieldDescriptor fieldDescriptor = descriptor.getFields().get(i);
+        @Nullable Object value = messageValueFromFieldValue(fieldDescriptor, 
cell.getV());
+        if (value != null) {
+          builder.setField(fieldDescriptor, value);
+        }
+      }
+
+      return builder.build();
+    } else {
+      return messageFromMap(descriptor, tableRow);
+    }
+  }
+
   @VisibleForTesting
   static DescriptorProto descriptorSchemaFromTableSchema(TableSchema 
tableSchema) {
     return descriptorSchemaFromTableFieldSchemas(tableSchema.getFields());
@@ -200,11 +224,13 @@ public class TableRowToStorageApiProto {
     }
 
     if (fieldDescriptor.getType() == FieldDescriptor.Type.MESSAGE) {
-      if (jsonBQValue instanceof AbstractMap) {
+      if (jsonBQValue instanceof TableRow) {
+        TableRow tableRow = (TableRow) jsonBQValue;
+        return messageFromTableRow(fieldDescriptor.getMessageType(), tableRow);
+      } else if (jsonBQValue instanceof AbstractMap) {
         // This will handle nested rows.
-        TableRow tr = new TableRow();
-        tr.putAll((AbstractMap<String, Object>) jsonBQValue);
-        return messageFromTableRow(fieldDescriptor.getMessageType(), tr);
+        AbstractMap<String, Object> map = ((AbstractMap<String, Object>) 
jsonBQValue);
+        return messageFromMap(fieldDescriptor.getMessageType(), map);
       } else {
         throw new RuntimeException("Unexpected value " + jsonBQValue + " 
Expected a JSON map.");
       }
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
index 7b1f89e..00040c1 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.io.gcp.bigquery;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import com.google.api.services.bigquery.model.TableCell;
 import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
@@ -38,6 +39,7 @@ import java.util.stream.Collectors;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions;
 import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -49,11 +51,40 @@ import org.junit.runners.JUnit4;
 })
 /** Unit tests for {@link 
org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto}. */
 public class TableRowToStorageApiProtoTest {
+  // Schemas we test.
+  // The TableRow class has special semantics for fields named "f". To ensure 
we handel them
+  // properly, we test schemas
+  // both with and without a field named "f".
   private static final TableSchema BASE_TABLE_SCHEMA =
       new TableSchema()
           .setFields(
               ImmutableList.<TableFieldSchema>builder()
                   .add(new 
TableFieldSchema().setType("STRING").setName("stringValue"))
+                  .add(new TableFieldSchema().setType("STRING").setName("f"))
+                  .add(new 
TableFieldSchema().setType("BYTES").setName("bytesValue"))
+                  .add(new 
TableFieldSchema().setType("INT64").setName("int64Value"))
+                  .add(new 
TableFieldSchema().setType("INTEGER").setName("intValue"))
+                  .add(new 
TableFieldSchema().setType("FLOAT64").setName("float64Value"))
+                  .add(new 
TableFieldSchema().setType("FLOAT").setName("floatValue"))
+                  .add(new 
TableFieldSchema().setType("BOOL").setName("boolValue"))
+                  .add(new 
TableFieldSchema().setType("BOOLEAN").setName("booleanValue"))
+                  .add(new 
TableFieldSchema().setType("TIMESTAMP").setName("timestampValue"))
+                  .add(new 
TableFieldSchema().setType("TIME").setName("timeValue"))
+                  .add(new 
TableFieldSchema().setType("DATETIME").setName("datetimeValue"))
+                  .add(new 
TableFieldSchema().setType("DATE").setName("dateValue"))
+                  .add(new 
TableFieldSchema().setType("NUMERIC").setName("numericValue"))
+                  .add(
+                      new TableFieldSchema()
+                          .setType("STRING")
+                          .setMode("REPEATED")
+                          .setName("arrayValue"))
+                  .build());
+
+  private static final TableSchema BASE_TABLE_SCHEMA_NO_F =
+      new TableSchema()
+          .setFields(
+              ImmutableList.<TableFieldSchema>builder()
+                  .add(new 
TableFieldSchema().setType("STRING").setName("stringValue"))
                   .add(new 
TableFieldSchema().setType("BYTES").setName("bytesValue"))
                   .add(new 
TableFieldSchema().setType("INT64").setName("int64Value"))
                   .add(new 
TableFieldSchema().setType("INTEGER").setName("intValue"))
@@ -84,6 +115,115 @@ public class TableRowToStorageApiProtoTest {
                   .build())
           .addField(
               FieldDescriptorProto.newBuilder()
+                  .setName("f")
+                  .setNumber(2)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("bytesvalue")
+                  .setNumber(3)
+                  .setType(Type.TYPE_BYTES)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("int64value")
+                  .setNumber(4)
+                  .setType(Type.TYPE_INT64)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("intvalue")
+                  .setNumber(5)
+                  .setType(Type.TYPE_INT64)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("float64value")
+                  .setNumber(6)
+                  .setType(Type.TYPE_DOUBLE)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("floatvalue")
+                  .setNumber(7)
+                  .setType(Type.TYPE_DOUBLE)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("boolvalue")
+                  .setNumber(8)
+                  .setType(Type.TYPE_BOOL)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("booleanvalue")
+                  .setNumber(9)
+                  .setType(Type.TYPE_BOOL)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("timestampvalue")
+                  .setNumber(10)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("timevalue")
+                  .setNumber(11)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("datetimevalue")
+                  .setNumber(12)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("datevalue")
+                  .setNumber(13)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("numericvalue")
+                  .setNumber(14)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("arrayvalue")
+                  .setNumber(15)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_REPEATED)
+                  .build())
+          .build();
+
+  private static final DescriptorProto BASE_TABLE_SCHEMA_NO_F_PROTO =
+      DescriptorProto.newBuilder()
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName("stringvalue")
+                  .setNumber(1)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
                   .setName("bytesvalue")
                   .setNumber(2)
                   .setType(Type.TYPE_BYTES)
@@ -155,7 +295,7 @@ public class TableRowToStorageApiProtoTest {
           .addField(
               FieldDescriptorProto.newBuilder()
                   .setName("datevalue")
-                  .setNumber(12)
+                  .setNumber(2)
                   .setType(Type.TYPE_STRING)
                   .setLabel(Label.LABEL_OPTIONAL)
                   .build())
@@ -174,7 +314,6 @@ public class TableRowToStorageApiProtoTest {
                   .setLabel(Label.LABEL_REPEATED)
                   .build())
           .build();
-
   private static final TableSchema NESTED_TABLE_SCHEMA =
       new TableSchema()
           .setFields(
@@ -189,6 +328,16 @@ public class TableRowToStorageApiProtoTest {
                           .setType("RECORD")
                           .setName("nestedValue2")
                           .setFields(BASE_TABLE_SCHEMA.getFields()))
+                  .add(
+                      new TableFieldSchema()
+                          .setType("STRUCT")
+                          .setName("nestedValueNoF1")
+                          .setFields(BASE_TABLE_SCHEMA_NO_F.getFields()))
+                  .add(
+                      new TableFieldSchema()
+                          .setType("RECORD")
+                          .setName("nestedValueNoF2")
+                          .setFields(BASE_TABLE_SCHEMA_NO_F.getFields()))
                   .build());
 
   @Test
@@ -214,6 +363,10 @@ public class TableRowToStorageApiProtoTest {
         BASE_TABLE_SCHEMA_PROTO.getFieldList().stream()
             .collect(
                 Collectors.toMap(FieldDescriptorProto::getName, 
FieldDescriptorProto::getType));
+    Map<String, Type> expectedBaseTypesNoF =
+        BASE_TABLE_SCHEMA_NO_F_PROTO.getFieldList().stream()
+            .collect(
+                Collectors.toMap(FieldDescriptorProto::getName, 
FieldDescriptorProto::getType));
 
     Map<String, Type> types =
         descriptor.getFieldList().stream()
@@ -223,12 +376,12 @@ public class TableRowToStorageApiProtoTest {
         descriptor.getFieldList().stream()
             .collect(
                 Collectors.toMap(FieldDescriptorProto::getName, 
FieldDescriptorProto::getTypeName));
-    assertEquals(2, types.size());
+    assertEquals(4, types.size());
 
     Map<String, DescriptorProto> nestedTypes =
         descriptor.getNestedTypeList().stream()
             .collect(Collectors.toMap(DescriptorProto::getName, 
Functions.identity()));
-    assertEquals(2, nestedTypes.size());
+    assertEquals(4, nestedTypes.size());
     assertEquals(Type.TYPE_MESSAGE, types.get("nestedvalue1"));
     String nestedTypeName1 = typeNames.get("nestedvalue1");
     Map<String, Type> nestedTypes1 =
@@ -244,10 +397,47 @@ public class TableRowToStorageApiProtoTest {
             .collect(
                 Collectors.toMap(FieldDescriptorProto::getName, 
FieldDescriptorProto::getType));
     assertEquals(expectedBaseTypes, nestedTypes2);
+
+    assertEquals(Type.TYPE_MESSAGE, types.get("nestedvaluenof1"));
+    String nestedTypeNameNoF1 = typeNames.get("nestedvaluenof1");
+    Map<String, Type> nestedTypesNoF1 =
+        nestedTypes.get(nestedTypeNameNoF1).getFieldList().stream()
+            .collect(
+                Collectors.toMap(FieldDescriptorProto::getName, 
FieldDescriptorProto::getType));
+    assertEquals(expectedBaseTypesNoF, nestedTypesNoF1);
+    assertEquals(Type.TYPE_MESSAGE, types.get("nestedvaluenof2"));
+    String nestedTypeNameNoF2 = typeNames.get("nestedvaluenof2");
+    Map<String, Type> nestedTypesNoF2 =
+        nestedTypes.get(nestedTypeNameNoF2).getFieldList().stream()
+            .collect(
+                Collectors.toMap(FieldDescriptorProto::getName, 
FieldDescriptorProto::getType));
+    assertEquals(expectedBaseTypesNoF, nestedTypesNoF2);
   }
 
   private static final TableRow BASE_TABLE_ROW =
       new TableRow()
+          .setF(
+              Lists.newArrayList(
+                  new TableCell().setV("string"),
+                  new TableCell().setV("fff"),
+                  new TableCell()
+                      .setV(
+                          
BaseEncoding.base64().encode("string".getBytes(StandardCharsets.UTF_8))),
+                  new TableCell().setV("42"),
+                  new TableCell().setV("43"),
+                  new TableCell().setV("2.8168"),
+                  new TableCell().setV("2.817"),
+                  new TableCell().setV("true"),
+                  new TableCell().setV("true"),
+                  new TableCell().setV("43"),
+                  new TableCell().setV("00:52:07[.123]|[.123456] UTC"),
+                  new TableCell().setV("2019-08-16 00:52:07[.123]|[.123456] 
UTC"),
+                  new TableCell().setV("2019-08-16"),
+                  new TableCell().setV("23.4"),
+                  new TableCell().setV(ImmutableList.of("hello", "goodbye"))));
+
+  private static final TableRow BASE_TABLE_ROW_NO_F =
+      new TableRow()
           .set("stringValue", "string")
           .set(
               "bytesValue", 
BaseEncoding.base64().encode("string".getBytes(StandardCharsets.UTF_8)))
@@ -267,6 +457,7 @@ public class TableRowToStorageApiProtoTest {
   private static final Map<String, Object> BASE_ROW_EXPECTED_PROTO_VALUES =
       ImmutableMap.<String, Object>builder()
           .put("stringvalue", "string")
+          .put("f", "fff")
           .put("bytesvalue", 
ByteString.copyFrom("string".getBytes(StandardCharsets.UTF_8)))
           .put("int64value", (long) 42)
           .put("intvalue", (long) 43)
@@ -282,28 +473,62 @@ public class TableRowToStorageApiProtoTest {
           .put("arrayvalue", ImmutableList.of("hello", "goodbye"))
           .build();
 
-  private void assertBaseRecord(DynamicMessage msg) {
+  private static final Map<String, Object> BASE_ROW_NO_F_EXPECTED_PROTO_VALUES 
=
+      ImmutableMap.<String, Object>builder()
+          .put("stringvalue", "string")
+          .put("bytesvalue", 
ByteString.copyFrom("string".getBytes(StandardCharsets.UTF_8)))
+          .put("int64value", (long) 42)
+          .put("intvalue", (long) 43)
+          .put("float64value", (double) 2.8168)
+          .put("floatvalue", (double) 2.817)
+          .put("boolvalue", true)
+          .put("booleanvalue", true)
+          .put("timestampvalue", "43")
+          .put("timevalue", "00:52:07[.123]|[.123456] UTC")
+          .put("datetimevalue", "2019-08-16 00:52:07[.123]|[.123456] UTC")
+          .put("datevalue", "2019-08-16")
+          .put("numericvalue", "23.4")
+          .put("arrayvalue", ImmutableList.of("hello", "goodbye"))
+          .build();
+
+  private void assertBaseRecord(DynamicMessage msg, boolean withF) {
     Map<String, Object> recordFields =
         msg.getAllFields().entrySet().stream()
             .collect(
                 Collectors.toMap(entry -> entry.getKey().getName(), entry -> 
entry.getValue()));
-    assertEquals(BASE_ROW_EXPECTED_PROTO_VALUES, recordFields);
+    assertEquals(
+        withF ? BASE_ROW_EXPECTED_PROTO_VALUES : 
BASE_ROW_NO_F_EXPECTED_PROTO_VALUES, recordFields);
   }
 
   @Test
   public void testMessageFromTableRow() throws Exception {
     TableRow tableRow =
-        new TableRow().set("nestedValue1", BASE_TABLE_ROW).set("nestedValue2", 
BASE_TABLE_ROW);
+        new TableRow()
+            .set("nestedValue1", BASE_TABLE_ROW)
+            .set("nestedValue2", BASE_TABLE_ROW)
+            .set("nestedvalueNoF1", BASE_TABLE_ROW_NO_F)
+            .set("nestedvalueNoF2", BASE_TABLE_ROW_NO_F);
+
     Descriptor descriptor =
         
TableRowToStorageApiProto.getDescriptorFromTableSchema(NESTED_TABLE_SCHEMA);
     DynamicMessage msg = 
TableRowToStorageApiProto.messageFromTableRow(descriptor, tableRow);
-    assertEquals(2, msg.getAllFields().size());
+    assertEquals(4, msg.getAllFields().size());
 
     Map<String, FieldDescriptor> fieldDescriptors =
         descriptor.getFields().stream()
             .collect(Collectors.toMap(FieldDescriptor::getName, 
Functions.identity()));
-    assertBaseRecord((DynamicMessage) 
msg.getField(fieldDescriptors.get("nestedvalue1")));
-    assertBaseRecord((DynamicMessage) 
msg.getField(fieldDescriptors.get("nestedvalue2")));
+    assertBaseRecord((DynamicMessage) 
msg.getField(fieldDescriptors.get("nestedvalue1")), true);
+    assertBaseRecord((DynamicMessage) 
msg.getField(fieldDescriptors.get("nestedvalue2")), true);
+    assertBaseRecord((DynamicMessage) 
msg.getField(fieldDescriptors.get("nestedvaluenof1")), false);
+    assertBaseRecord((DynamicMessage) 
msg.getField(fieldDescriptors.get("nestedvaluenof2")), false);
+  }
+
+  @Test
+  public void testMessageWithFFromTableRow() throws Exception {
+    Descriptor descriptor =
+        
TableRowToStorageApiProto.getDescriptorFromTableSchema(BASE_TABLE_SCHEMA);
+    DynamicMessage msg = 
TableRowToStorageApiProto.messageFromTableRow(descriptor, BASE_TABLE_ROW);
+    assertBaseRecord(msg, true);
   }
 
   private static final TableSchema REPEATED_MESSAGE_SCHEMA =
@@ -319,6 +544,16 @@ public class TableRowToStorageApiProtoTest {
                       .setType("RECORD")
                       .setName("repeated2")
                       .setFields(BASE_TABLE_SCHEMA.getFields())
+                      .setMode("REPEATED"),
+                  new TableFieldSchema()
+                      .setType("STRUCT")
+                      .setName("repeatednof1")
+                      .setFields(BASE_TABLE_SCHEMA_NO_F.getFields())
+                      .setMode("REPEATED"),
+                  new TableFieldSchema()
+                      .setType("RECORD")
+                      .setName("repeatednof2")
+                      .setFields(BASE_TABLE_SCHEMA_NO_F.getFields())
                       .setMode("REPEATED")));
 
   @Test
@@ -326,11 +561,13 @@ public class TableRowToStorageApiProtoTest {
     TableRow repeatedRow =
         new TableRow()
             .set("repeated1", ImmutableList.of(BASE_TABLE_ROW, BASE_TABLE_ROW))
-            .set("repeated2", ImmutableList.of(BASE_TABLE_ROW, 
BASE_TABLE_ROW));
+            .set("repeated2", ImmutableList.of(BASE_TABLE_ROW, BASE_TABLE_ROW))
+            .set("repeatednof1", ImmutableList.of(BASE_TABLE_ROW_NO_F, 
BASE_TABLE_ROW_NO_F))
+            .set("repeatednof2", ImmutableList.of(BASE_TABLE_ROW_NO_F, 
BASE_TABLE_ROW_NO_F));
     Descriptor descriptor =
         
TableRowToStorageApiProto.getDescriptorFromTableSchema(REPEATED_MESSAGE_SCHEMA);
     DynamicMessage msg = 
TableRowToStorageApiProto.messageFromTableRow(descriptor, repeatedRow);
-    assertEquals(2, msg.getAllFields().size());
+    assertEquals(4, msg.getAllFields().size());
 
     Map<String, FieldDescriptor> fieldDescriptors =
         descriptor.getFields().stream()
@@ -338,19 +575,36 @@ public class TableRowToStorageApiProtoTest {
     List<DynamicMessage> repeated1 =
         (List<DynamicMessage>) msg.getField(fieldDescriptors.get("repeated1"));
     assertEquals(2, repeated1.size());
-    assertBaseRecord(repeated1.get(0));
-    assertBaseRecord(repeated1.get(1));
+    assertBaseRecord(repeated1.get(0), true);
+    assertBaseRecord(repeated1.get(1), true);
 
     List<DynamicMessage> repeated2 =
         (List<DynamicMessage>) msg.getField(fieldDescriptors.get("repeated2"));
     assertEquals(2, repeated2.size());
-    assertBaseRecord(repeated2.get(0));
-    assertBaseRecord(repeated2.get(1));
+    assertBaseRecord(repeated2.get(0), true);
+    assertBaseRecord(repeated2.get(1), true);
+
+    List<DynamicMessage> repeatednof1 =
+        (List<DynamicMessage>) 
msg.getField(fieldDescriptors.get("repeatednof1"));
+    assertEquals(2, repeatednof1.size());
+    assertBaseRecord(repeatednof1.get(0), false);
+    assertBaseRecord(repeatednof1.get(1), false);
+
+    List<DynamicMessage> repeatednof2 =
+        (List<DynamicMessage>) 
msg.getField(fieldDescriptors.get("repeatednof2"));
+    assertEquals(2, repeatednof2.size());
+    assertBaseRecord(repeatednof2.get(0), false);
+    assertBaseRecord(repeatednof2.get(1), false);
   }
 
   @Test
   public void testNullRepeatedDescriptorFromTableSchema() throws Exception {
-    TableRow repeatedRow = new TableRow().set("repeated1", 
null).set("repeated2", null);
+    TableRow repeatedRow =
+        new TableRow()
+            .set("repeated1", null)
+            .set("repeated2", null)
+            .set("repeatednof1", null)
+            .set("repeatednof2", null);
     Descriptor descriptor =
         
TableRowToStorageApiProto.getDescriptorFromTableSchema(REPEATED_MESSAGE_SCHEMA);
     DynamicMessage msg = 
TableRowToStorageApiProto.messageFromTableRow(descriptor, repeatedRow);
@@ -364,5 +618,11 @@ public class TableRowToStorageApiProtoTest {
     List<DynamicMessage> repeated2 =
         (List<DynamicMessage>) msg.getField(fieldDescriptors.get("repeated2"));
     assertTrue(repeated2.isEmpty());
+    List<DynamicMessage> repeatednof1 =
+        (List<DynamicMessage>) 
msg.getField(fieldDescriptors.get("repeatednof1"));
+    assertTrue(repeatednof1.isEmpty());
+    List<DynamicMessage> repeatednof2 =
+        (List<DynamicMessage>) 
msg.getField(fieldDescriptors.get("repeatednof2"));
+    assertTrue(repeatednof2.isEmpty());
   }
 }

Reply via email to