yirutang commented on code in PR #24145:
URL: https://github.com/apache/beam/pull/24145#discussion_r1067582202


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java:
##########
@@ -512,6 +581,18 @@ String retrieveErrorDetails(Iterable<AppendRowsContext> 
failedContext) {
             .map(StackTraceElement::toString)
             .collect(Collectors.joining("\n"));
       }
+
+      void postFlush() {
+        // If we got a response indicating an updated schema, recreate the 
client.
+        if (updatedTableSchema != null
+            && this.appendClientInfo != null
+            && this.appendClientInfo.hasSchemaChanged(updatedTableSchema)) {

Review Comment:
   Not sure if this is expensive to perform, theoretically it is not needed 
since when setting updatedTableSchema it is already checked. 
   
   We also provided something here:
   
https://github.com/googleapis/java-bigquerystorage/blob/main/google-cloud-bigquerystorage/src/main/java/com/google/cloud/bigquery/storage/v1/StreamWriter.java#L157
 
   
   If that field is not null, it means a new schema appeared. We do the 
comparison based on timestamp.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -652,6 +692,18 @@ public void process(
         }
         appendSplitDistribution.update(numAppends);
 
+        if (updatedSchemaReturned.get() != null) {

Review Comment:
   Wondering if you can directly use the one on StreamWriter:
   
https://github.com/googleapis/java-bigquerystorage/blob/main/google-cloud-bigquerystorage/src/main/java/com/google/cloud/bigquery/storage/v1/StreamWriter.java#L157



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