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

xqhu 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 c0408956a59 handle flexible BigQuery column names (#35631)
c0408956a59 is described below

commit c0408956a595943ccabffca3391ba516f2146035
Author: Reuven Lax <[email protected]>
AuthorDate: Sat Jul 19 10:24:39 2025 -0700

    handle flexible BigQuery column names (#35631)
    
    * handle flexible BigQuery column names
    
    * add more tests
    
    * add integration test
---
 .../io/gcp/bigquery/TableRowToStorageApiProto.java | 42 ++++++++++--
 .../sdk/io/gcp/bigquery/BigQueryIOWriteTest.java   | 27 +++++---
 .../gcp/bigquery/TableRowToStorageApiProtoIT.java  | 29 +++++---
 .../bigquery/TableRowToStorageApiProtoTest.java    | 78 +++++++++++++++++++++-
 4 files changed, 150 insertions(+), 26 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 37b09349646..bf9c4c28bc1 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
@@ -21,10 +21,13 @@ import static java.util.stream.Collectors.toList;
 
 import com.google.api.services.bigquery.model.TableCell;
 import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.bigquery.storage.v1.AnnotationsProto;
 import com.google.cloud.bigquery.storage.v1.BigDecimalByteStringEncoder;
+import com.google.cloud.bigquery.storage.v1.BigQuerySchemaUtil;
 import com.google.cloud.bigquery.storage.v1.TableFieldSchema;
 import com.google.cloud.bigquery.storage.v1.TableSchema;
 import com.google.protobuf.ByteString;
+import com.google.protobuf.DescriptorProtos;
 import com.google.protobuf.DescriptorProtos.DescriptorProto;
 import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
 import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
@@ -487,7 +490,11 @@ public class TableRowToStorageApiProto {
     DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor);
     for (final Map.Entry<String, Object> entry : map.entrySet()) {
       String key = entry.getKey().toLowerCase();
-      @Nullable FieldDescriptor fieldDescriptor = 
descriptor.findFieldByName(key);
+      String protoFieldName =
+          BigQuerySchemaUtil.isProtoCompatible(key)
+              ? key
+              : BigQuerySchemaUtil.generatePlaceholderFieldName(key);
+      @Nullable FieldDescriptor fieldDescriptor = 
descriptor.findFieldByName(protoFieldName);
       if (fieldDescriptor == null) {
         if (unknownFields != null) {
           unknownFields.set(key, entry.getValue());
@@ -724,9 +731,18 @@ public class TableRowToStorageApiProto {
     return TableSchema.newBuilder().addAllFields(tableFields).build();
   }
 
+  private static String fieldNameFromProtoFieldDescriptor(FieldDescriptor 
fieldDescriptor) {
+    if 
(fieldDescriptor.getOptions().hasExtension(AnnotationsProto.columnName)) {
+      return 
fieldDescriptor.getOptions().getExtension(AnnotationsProto.columnName);
+    } else {
+      return fieldDescriptor.getName();
+    }
+  }
+
   static TableFieldSchema tableFieldSchemaFromDescriptorField(FieldDescriptor 
fieldDescriptor) {
     TableFieldSchema.Builder tableFieldSchemaBuilder = 
TableFieldSchema.newBuilder();
-    tableFieldSchemaBuilder = 
tableFieldSchemaBuilder.setName(fieldDescriptor.getName());
+    tableFieldSchemaBuilder =
+        
tableFieldSchemaBuilder.setName(fieldNameFromProtoFieldDescriptor(fieldDescriptor));
 
     switch (fieldDescriptor.getType()) {
       case MESSAGE:
@@ -809,8 +825,21 @@ public class TableRowToStorageApiProto {
           "Reserved field name " + fieldSchema.getName() + " in user schema.");
     }
     FieldDescriptorProto.Builder fieldDescriptorBuilder = 
FieldDescriptorProto.newBuilder();
-    fieldDescriptorBuilder = 
fieldDescriptorBuilder.setName(fieldSchema.getName().toLowerCase());
+    final String fieldName = fieldSchema.getName().toLowerCase();
+    fieldDescriptorBuilder = fieldDescriptorBuilder.setName(fieldName);
     fieldDescriptorBuilder = fieldDescriptorBuilder.setNumber(fieldNumber);
+    if (!BigQuerySchemaUtil.isProtoCompatible(fieldName)) {
+      fieldDescriptorBuilder =
+          fieldDescriptorBuilder.setName(
+              BigQuerySchemaUtil.generatePlaceholderFieldName(fieldName));
+
+      Message.Builder fieldOptionBuilder = 
DescriptorProtos.FieldOptions.newBuilder();
+      fieldOptionBuilder =
+          
fieldOptionBuilder.setField(AnnotationsProto.columnName.getDescriptor(), 
fieldName);
+      fieldDescriptorBuilder =
+          fieldDescriptorBuilder.setOptions(
+              (DescriptorProtos.FieldOptions) fieldOptionBuilder.build());
+    }
     switch (fieldSchema.getType()) {
       case STRUCT:
         DescriptorProto nested =
@@ -1113,12 +1142,13 @@ public class TableRowToStorageApiProto {
     for (Map.Entry<FieldDescriptor, Object> field : 
message.getAllFields().entrySet()) {
       StringBuilder fullName = new StringBuilder();
       FieldDescriptor fieldDescriptor = field.getKey();
-      fullName = fullName.append(namePrefix).append(fieldDescriptor.getName());
+      String fieldName = fieldNameFromProtoFieldDescriptor(fieldDescriptor);
+      fullName = fullName.append(namePrefix).append(fieldName);
       Object fieldValue = field.getValue();
       if ((includeCdcColumns || 
!StorageApiCDC.COLUMNS.contains(fullName.toString()))
-          && includeField.test(fieldDescriptor.getName())) {
+          && includeField.test(fieldName)) {
         tableRow.put(
-            fieldDescriptor.getName(),
+            fieldName,
             jsonValueFromMessageValue(
                 fieldDescriptor, fieldValue, true, includeField, 
fullName.append(".").toString()));
       }
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
index 88d7d2e7326..5f4b9c7c29e 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
@@ -1452,9 +1452,19 @@ public class BigQueryIOWriteTest implements Serializable 
{
       assumeTrue(!useStorageApiApproximate);
       assumeTrue(useStreaming);
     }
+    TableSchema tableSchema =
+        new TableSchema()
+            .setFields(
+                ImmutableList.of(
+                    new 
TableFieldSchema().setName("number").setType("INTEGER"),
+                    // Make sure to exercise the name override by specifying 
an illegal proto field
+                    // name.
+                    new 
TableFieldSchema().setName("123_number").setType("INTEGER")));
+
     List<TableRow> elements = Lists.newArrayList();
     for (int i = 0; i < 30; ++i) {
-      elements.add(new TableRow().set("number", String.valueOf(i)));
+      String value = String.valueOf(i);
+      elements.add(new TableRow().set("number", value).set("123_number", 
value));
     }
 
     TestStream<TableRow> testStream =
@@ -1473,11 +1483,7 @@ public class BigQueryIOWriteTest implements Serializable 
{
         BigQueryIO.writeTableRows()
             .to("project-id:dataset-id.table-id")
             
.withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-            .withSchema(
-                new TableSchema()
-                    .setFields(
-                        ImmutableList.of(
-                            new 
TableFieldSchema().setName("number").setType("INTEGER"))))
+            .withSchema(tableSchema)
             .withTestServices(fakeBqServices)
             .withoutValidation();
 
@@ -2398,7 +2404,8 @@ public class BigQueryIOWriteTest implements Serializable {
                     new TableFieldSchema().setName("name").setType("STRING"),
                     new 
TableFieldSchema().setName("number").setType("INTEGER"),
                     new TableFieldSchema().setName("req").setType("STRING"),
-                    new 
TableFieldSchema().setName("double_number").setType("INTEGER")));
+                    new 
TableFieldSchema().setName("double_number").setType("INTEGER"),
+                    new 
TableFieldSchema().setName("12_special_name").setType("STRING")));
     fakeDatasetService.createTable(new 
Table().setTableReference(tableRef).setSchema(tableSchema));
 
     LongFunction<TableRow> getRowSet =
@@ -2408,7 +2415,8 @@ public class BigQueryIOWriteTest implements Serializable {
                   new TableRow()
                       .set("name", "name" + i)
                       .set("number", Long.toString(i))
-                      .set("double_number", Long.toString(i * 2));
+                      .set("double_number", Long.toString(i * 2))
+                      .set("12_special_name", "name" + i);
               if (i <= 5) {
                 row = row.set("req", "foo");
               }
@@ -2424,7 +2432,8 @@ public class BigQueryIOWriteTest implements Serializable {
                             new TableCell().setV(Long.toString(i)),
                             new TableCell().setV("name" + i),
                             new TableCell().setV(i > 5 ? null : "foo"),
-                            new TableCell().setV(Long.toString(i * 2))));
+                            new TableCell().setV(Long.toString(i * 2)),
+                            new TableCell().setV("name" + i)));
 
     LongFunction<TableRow> getRow = useSet ? getRowSet : getRowSetF;
 
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java
index f28ae588a5e..1ae691cb7e9 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java
@@ -88,6 +88,8 @@ public class TableRowToStorageApiProtoIT {
                           .setType("BYTES")
                           .setMode("REPEATED")
                           .setName("arrayValue"))
+                  .add(
+                      new 
TableFieldSchema().setType("STRING").setName("123_IllegalProtoFieldName"))
                   .build());
 
   private static final List<Object> REPEATED_BYTES =
@@ -113,7 +115,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", "2019-08-16")
           .set("numericValue", "23.4")
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES);
+          .set("arrayValue", REPEATED_BYTES)
+          .set("123_IllegalProtoFieldName", "string");
 
   private static final TableRow BASE_TABLE_ROW_JODA_TIME =
       new TableRow()
@@ -131,7 +134,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", org.joda.time.LocalDate.parse("2019-08-16"))
           .set("numericValue", new BigDecimal("23.4"))
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES);
+          .set("arrayValue", REPEATED_BYTES)
+          .set("123_IllegalProtoFieldName", "string");
 
   private static final TableRow BASE_TABLE_ROW_JAVA_TIME =
       new TableRow()
@@ -149,7 +153,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", LocalDate.parse("2019-08-16"))
           .set("numericValue", new BigDecimal("23.4"))
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES);
+          .set("arrayValue", REPEATED_BYTES)
+          .set("123_IllegalProtoFieldName", "string");
 
   private static final TableRow BASE_TABLE_ROW_NUM_TIME =
       new TableRow()
@@ -167,7 +172,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", 18124)
           .set("numericValue", new BigDecimal("23.4"))
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES);
+          .set("arrayValue", REPEATED_BYTES)
+          .set("123_IllegalProtoFieldName", "string");
 
   @SuppressWarnings({
     "FloatingPointLiteralPrecision" // 
https://github.com/apache/beam/issues/23666
@@ -188,7 +194,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", 18124)
           .set("numericValue", 23.4)
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES);
+          .set("arrayValue", REPEATED_BYTES)
+          .set("123_IllegalProtoFieldName", "string");
 
   private static final TableRow BASE_TABLE_ROW_NULL =
       new TableRow()
@@ -230,7 +237,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", "2019-08-16")
           .set("numericValue", "23.4")
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES_EXPECTED);
+          .set("arrayValue", REPEATED_BYTES_EXPECTED)
+          .set("123_IllegalProtoFieldName", "string");
 
   // joda is up to millisecond precision, expect truncation
   private static final TableRow BASE_TABLE_ROW_JODA_EXPECTED =
@@ -250,7 +258,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", "2019-08-16")
           .set("numericValue", "23.4")
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES_EXPECTED);
+          .set("arrayValue", REPEATED_BYTES_EXPECTED)
+          .set("123_IllegalProtoFieldName", "string");
 
   private static final TableRow BASE_TABLE_ROW_NUM_EXPECTED =
       new TableRow()
@@ -269,7 +278,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", "2019-08-16")
           .set("numericValue", "23.4")
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES_EXPECTED);
+          .set("arrayValue", REPEATED_BYTES_EXPECTED)
+          .set("123_IllegalProtoFieldName", "string");
 
   private static final TableRow BASE_TABLE_ROW_FLOATS_EXPECTED =
       new TableRow()
@@ -288,7 +298,8 @@ public class TableRowToStorageApiProtoIT {
           .set("dateValue", "2019-08-16")
           .set("numericValue", "23.4")
           .set("bigNumericValue", "23334.4")
-          .set("arrayValue", REPEATED_BYTES_EXPECTED);
+          .set("arrayValue", REPEATED_BYTES_EXPECTED)
+          .set("123_IllegalProtoFieldName", "string");
 
   // only nonnull values are returned, null in arrayValue should be converted 
to empty list
   private static final TableRow BASE_TABLE_ROW_NULL_EXPECTED =
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 6e080d82de4..1a6b83c5ebd 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
@@ -27,8 +27,11 @@ 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;
+import com.google.cloud.bigquery.storage.v1.AnnotationsProto;
 import com.google.cloud.bigquery.storage.v1.BigDecimalByteStringEncoder;
+import com.google.cloud.bigquery.storage.v1.BigQuerySchemaUtil;
 import com.google.protobuf.ByteString;
+import com.google.protobuf.DescriptorProtos;
 import com.google.protobuf.DescriptorProtos.DescriptorProto;
 import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
 import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
@@ -119,6 +122,8 @@ public class TableRowToStorageApiProtoTest {
                           .setName("timestampValueSpaceTrailingZero"))
                   .add(new 
TableFieldSchema().setType("DATETIME").setName("datetimeValueSpace"))
                   .add(new 
TableFieldSchema().setType("TIMESTAMP").setName("timestampValueMaximum"))
+                  .add(
+                      new 
TableFieldSchema().setType("STRING").setName("123_IllegalProtoFieldName"))
                   .build());
 
   private static final TableSchema BASE_TABLE_SCHEMA_NO_F =
@@ -169,6 +174,8 @@ public class TableRowToStorageApiProtoTest {
                           .setName("timestampValueSpaceTrailingZero"))
                   .add(new 
TableFieldSchema().setType("DATETIME").setName("datetimeValueSpace"))
                   .add(new 
TableFieldSchema().setType("TIMESTAMP").setName("timestampValueMaximum"))
+                  .add(
+                      new 
TableFieldSchema().setType("STRING").setName("123_IllegalProtoFieldName"))
                   .build());
 
   private static final DescriptorProto BASE_TABLE_SCHEMA_PROTO_DESCRIPTOR =
@@ -369,6 +376,19 @@ public class TableRowToStorageApiProtoTest {
                   .setType(Type.TYPE_INT64)
                   .setLabel(Label.LABEL_OPTIONAL)
                   .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName(
+                      
BigQuerySchemaUtil.generatePlaceholderFieldName("123_illegalprotofieldname"))
+                  .setNumber(29)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .setOptions(
+                      DescriptorProtos.FieldOptions.newBuilder()
+                          .setField(
+                              AnnotationsProto.columnName.getDescriptor(),
+                              "123_illegalprotofieldname"))
+                  .build())
           .build();
 
   private static final com.google.cloud.bigquery.storage.v1.TableSchema 
BASE_TABLE_PROTO_SCHEMA =
@@ -513,6 +533,11 @@ public class TableRowToStorageApiProtoTest {
                   .setName("timestampvaluemaximum")
                   
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
                   .build())
+          .addFields(
+              
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+                  .setName("123_illegalprotofieldname")
+                  
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.STRING)
+                  .build())
           .build();
 
   private static final DescriptorProto BASE_TABLE_SCHEMA_NO_F_PROTO =
@@ -706,6 +731,19 @@ public class TableRowToStorageApiProtoTest {
                   .setType(Type.TYPE_INT64)
                   .setLabel(Label.LABEL_OPTIONAL)
                   .build())
+          .addField(
+              FieldDescriptorProto.newBuilder()
+                  .setName(
+                      
BigQuerySchemaUtil.generatePlaceholderFieldName("123_illegalprotofieldname"))
+                  .setNumber(28)
+                  .setType(Type.TYPE_STRING)
+                  .setLabel(Label.LABEL_OPTIONAL)
+                  .setOptions(
+                      DescriptorProtos.FieldOptions.newBuilder()
+                          .setField(
+                              AnnotationsProto.columnName.getDescriptor(),
+                              "123_illegalprotofieldname"))
+                  .build())
           .build();
 
   private static final com.google.cloud.bigquery.storage.v1.TableSchema
@@ -846,6 +884,11 @@ public class TableRowToStorageApiProtoTest {
                       .setName("timestampvaluemaximum")
                       
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.INT64)
                       .build())
+              .addFields(
+                  
com.google.cloud.bigquery.storage.v1.TableFieldSchema.newBuilder()
+                      .setName("123_illegalprotofieldname")
+                      
.setType(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type.STRING)
+                      .build())
               .build();
   private static final TableSchema NESTED_TABLE_SCHEMA =
       new TableSchema()
@@ -1108,7 +1151,8 @@ public class TableRowToStorageApiProtoTest {
                   new TableCell().setV("1970-01-01 00:00:00.123"),
                   new TableCell().setV("1970-01-01 00:00:00.1230"),
                   new TableCell().setV("2019-08-16 00:52:07.123456"),
-                  new TableCell().setV("9999-12-31 23:59:59.999999Z")));
+                  new TableCell().setV("9999-12-31 23:59:59.999999Z"),
+                  new TableCell().setV("madeit")));
 
   private static final TableRow BASE_TABLE_ROW_NO_F =
       new TableRow()
@@ -1141,7 +1185,8 @@ public class TableRowToStorageApiProtoTest {
           .set("timestampValueSpaceMilli", "1970-01-01 00:00:00.123")
           .set("timestampValueSpaceTrailingZero", "1970-01-01 00:00:00.1230")
           .set("datetimeValueSpace", "2019-08-16 00:52:07.123456")
-          .set("timestampValueMaximum", "9999-12-31 23:59:59.999999Z");
+          .set("timestampValueMaximum", "9999-12-31 23:59:59.999999Z")
+          .set("123_illegalprotofieldname", "madeit");
 
   private static final Map<String, Object> BASE_ROW_EXPECTED_PROTO_VALUES =
       ImmutableMap.<String, Object>builder()
@@ -1182,8 +1227,16 @@ public class TableRowToStorageApiProtoTest {
           .put("timestampvaluespacetrailingzero", 123000L)
           .put("datetimevaluespace", 142111881387172416L)
           .put("timestampvaluemaximum", 253402300799999999L)
+          .put(
+              
BigQuerySchemaUtil.generatePlaceholderFieldName("123_illegalprotofieldname"),
+              "madeit")
           .build();
 
+  private static final Map<String, String> BASE_ROW_EXPECTED_NAME_OVERRIDES =
+      ImmutableMap.of(
+          
BigQuerySchemaUtil.generatePlaceholderFieldName("123_illegalprotofieldname"),
+          "123_illegalprotofieldname");
+
   private static final Map<String, Object> BASE_ROW_NO_F_EXPECTED_PROTO_VALUES 
=
       ImmutableMap.<String, Object>builder()
           .put("stringvalue", "string")
@@ -1222,15 +1275,36 @@ public class TableRowToStorageApiProtoTest {
           .put("timestampvaluespacetrailingzero", 123000L)
           .put("datetimevaluespace", 142111881387172416L)
           .put("timestampvaluemaximum", 253402300799999999L)
+          .put(
+              
BigQuerySchemaUtil.generatePlaceholderFieldName("123_illegalprotofieldname"),
+              "madeit")
           .build();
 
+  private static final Map<String, String> 
BASE_ROW_NO_F_EXPECTED_NAME_OVERRIDES =
+      ImmutableMap.of(
+          
BigQuerySchemaUtil.generatePlaceholderFieldName("123_illegalprotofieldname"),
+          "123_illegalprotofieldname");
+
   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()));
+
+    Map<String, String> overriddenNames =
+        msg.getAllFields().entrySet().stream()
+            .filter(entry -> 
entry.getKey().getOptions().hasExtension(AnnotationsProto.columnName))
+            .collect(
+                Collectors.toMap(
+                    entry -> entry.getKey().getName(),
+                    entry ->
+                        
entry.getKey().getOptions().getExtension(AnnotationsProto.columnName)));
+
     assertEquals(
         withF ? BASE_ROW_EXPECTED_PROTO_VALUES : 
BASE_ROW_NO_F_EXPECTED_PROTO_VALUES, recordFields);
+    assertEquals(
+        withF ? BASE_ROW_EXPECTED_NAME_OVERRIDES : 
BASE_ROW_NO_F_EXPECTED_NAME_OVERRIDES,
+        overriddenNames);
   }
 
   @Test

Reply via email to