ahmedabu98 commented on code in PR #38058:
URL: https://github.com/apache/beam/pull/38058#discussion_r3124871243


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java:
##########
@@ -4233,13 +4250,18 @@ private <DestinationT> WriteResult continueExpandTyped(
               !getIgnoreUnknownValues(),
               "ignoreUnknownValues not supported when using writeProtos."
                   + " Try setting withDirectWriteProtos(false)");
+          checkArgument(!useSchemaUpdate);
+

Review Comment:
   It's still there ^



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpgradeTableSchema.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.bigquery;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import com.google.cloud.bigquery.storage.v1.TableFieldSchema;
+import com.google.cloud.bigquery.storage.v1.TableSchema;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.DynamicMessage;
+import com.google.protobuf.Message;
+import com.google.protobuf.UnknownFieldSet;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.util.ThrowingSupplier;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.HashCode;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+public class UpgradeTableSchema {
+  public static TableRowToStorageApiProto.ErrorCollector newErrorCollector() {
+    return new TableRowToStorageApiProto.ErrorCollector(
+        e ->
+            (e instanceof TableRowToStorageApiProto.SchemaTooNarrowException
+                    && !((TableRowToStorageApiProto.SchemaTooNarrowException) 
e)
+                        .getMissingField()
+                        .isEmpty())
+                || e instanceof 
TableRowToStorageApiProto.SchemaMissingRequiredFieldException);
+  }
+
+  public static TableSchema getIncrementalSchema(
+      TableRowToStorageApiProto.ErrorCollector errorCollector, TableSchema 
oldSchema)
+      throws TableRowToStorageApiProto.SchemaDoesntMatchException {
+    // This isn't the most efficient, especially if we have deeply-nested 
schemas. However we don't
+    // expect to be
+    // upgrading schemas very regularly - if we do then we have other problems!
+    Map<String, LinkedHashMap<String, TableFieldSchema>> newFields = 
Maps.newHashMap();
+    Map<String, Set<String>> relaxedFields = Maps.newHashMap();
+
+    for (TableRowToStorageApiProto.SchemaConversionException 
schemaConversionException :
+        errorCollector.getExceptions()) {
+      if (schemaConversionException instanceof 
TableRowToStorageApiProto.SchemaTooNarrowException) {
+        TableRowToStorageApiProto.SchemaTooNarrowException e =
+            (TableRowToStorageApiProto.SchemaTooNarrowException) 
schemaConversionException;
+        List<String> components = 
Arrays.asList(e.getMissingField().toLowerCase().split("\\."));
+        String prefix = String.join(".", components.subList(0, 
components.size() - 1));
+        String name = components.get(components.size() - 1);
+        TableFieldSchema.Mode mode =
+            e.isRepeated() ? TableFieldSchema.Mode.REPEATED : 
TableFieldSchema.Mode.NULLABLE;
+        TableFieldSchema.Type type =
+            e.isStruct() ? TableFieldSchema.Type.STRUCT : 
TableFieldSchema.Type.STRING;
+        @Nullable
+        TableFieldSchema oldValue =
+            newFields
+                .computeIfAbsent(prefix, p -> Maps.newLinkedHashMap())
+                .put(
+                    name,
+                    TableFieldSchema.newBuilder()
+                        .setName(name)
+                        .setMode(mode)
+                        .setType(type)
+                        .build());

Review Comment:
   Was thinking about relaxing types, but I'm seeing now that 
`SchemaUpdateOption` doesn't have an option for that, so we can ignore this for 
now



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java:
##########
@@ -425,18 +523,41 @@ public interface ThrowingBiFunction<FirstInputT, 
SecondInputT, OutputT> {
                   })
               .put(
                   TableFieldSchema.Type.STRING,
-                  (schemaInformation, value) ->
-                      Preconditions.checkArgumentNotNull(value).toString())
+                  (fullName, value) -> 
Preconditions.checkArgumentNotNull(value).toString())
               .put(
                   TableFieldSchema.Type.JSON,
-                  (schemaInformation, value) ->
-                      Preconditions.checkArgumentNotNull(value).toString())
+                  (fullName, value) -> 
Preconditions.checkArgumentNotNull(value).toString())
               .put(
                   TableFieldSchema.Type.GEOGRAPHY,
-                  (schemaInformation, value) ->
-                      Preconditions.checkArgumentNotNull(value).toString())
+                  (fullName, value) -> 
Preconditions.checkArgumentNotNull(value).toString())
               .build();
 
+  static final HashFunction SCHEMA_HASH_FUNCTION = Hashing.goodFastHash(32);
+
+  public static byte[] tableSchemaHash(TableSchema tableSchema) {
+    return tableSchemaHash("", tableSchema.getFieldsList()).asBytes();
+  }
+
+  public static HashCode tableSchemaHash(String prefix, List<TableFieldSchema> 
fields) {
+    List<HashCode> hashCodes = Lists.newArrayList();
+    for (TableFieldSchema tableFieldSchema : fields) {
+      String name =
+          prefix.isEmpty()
+              ? tableFieldSchema.getName()
+              : String.join(".", prefix, tableFieldSchema.getName());
+      hashCodes.add(SCHEMA_HASH_FUNCTION.hashString(name.toLowerCase(), 
StandardCharsets.UTF_8));
+      hashCodes.add(
+          SCHEMA_HASH_FUNCTION.hashString(
+              tableFieldSchema.getType().toString().toLowerCase(), 
StandardCharsets.UTF_8));

Review Comment:
   is this more reliable?
   ```suggestion
         hashCodes.add(
             
SCHEMA_HASH_FUNCTION.hashInt(tableFieldSchema.getType().getNumber());
   ```



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java:
##########
@@ -89,6 +93,45 @@
  * with the Storage write API.
  */
 public class TableRowToStorageApiProto {
+
+  public static class ErrorCollector {
+    private final List<SchemaConversionException> exceptions = 
Lists.newArrayList();
+    private final Predicate<SchemaConversionException> shouldCollect;
+
+    public static final ErrorCollector DONT_COLLECT = new 
ErrorCollector(Predicates.alwaysFalse());
+
+    public ErrorCollector(Predicate<SchemaConversionException> shouldCollect) {
+      this.shouldCollect = shouldCollect;
+    }
+
+    // Returns true if the exception was collected.
+    void collect(SchemaConversionException exception) throws 
SchemaConversionException {

Review Comment:
   still there ^



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaUpdateHoldingFn.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.bigquery;
+
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.ExponentialBackOff;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.state.BagState;
+import org.apache.beam.sdk.state.CombiningState;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerSpec;
+import org.apache.beam.sdk.state.TimerSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.ShardedKey;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TimestampedValue;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * This is a stateful DoFn that buffers elements that triggered table schema 
update. Once the table
+ * schema has been updated, this reprocesses the messages and allows them to 
continue on through the
+ * sink.
+ */

Review Comment:
   Looks great, thanks!



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to