You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2023/01/04 00:26:47 UTC

[GitHub] [beam] prodriguezdefino commented on a diff in pull request #24145: Handle updates to table schema when using Storage API writes.

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


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -385,34 +399,57 @@ public void process(
 
       Supplier<String> getOrCreateStream =
           () -> getOrCreateStream(tableId, streamName, streamOffset, idleTimer, datasetService);
-      Function<Boolean, AppendClientInfo> getAppendClientInfo =
-          createAppendClient -> {
-            try {
-              @Nullable
-              TableSchema tableSchema =
-                  messageConverters
-                      .get(element.getKey().getKey(), dynamicDestinations, datasetService)
-                      .getTableSchema();
-              AppendClientInfo info =
-                  new AppendClientInfo(
-                      tableSchema,
-                      // Make sure that the client is always closed in a different thread to avoid
-                      // blocking.
-                      client -> runAsyncIgnoreFailure(closeWriterExecutor, client::close));
-              if (createAppendClient) {
-                info = info.createAppendClient(datasetService, getOrCreateStream, false);
-              }
-              return info;
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          };
-
       AtomicReference<AppendClientInfo> appendClientInfo =
           new AtomicReference<>(
-              APPEND_CLIENTS.get(element.getKey(), () -> getAppendClientInfo.apply(true)));
+              APPEND_CLIENTS.get(
+                  element.getKey(),
+                  () -> {
+                    @Nullable TableSchema tableSchema;
+                    if (autoUpdateSchema && updatedSchema.read() != null) {
+                      // We've seen an updated schema, so we use that.
+                      tableSchema = updatedSchema.read();
+                    } else {
+                      // Start off with the base schema. As we get notified of schema updates, we
+                      // will
+                      // update the
+                      // descriptor.
+                      tableSchema =
+                          messageConverters
+                              .get(element.getKey().getKey(), dynamicDestinations, datasetService)
+                              .getTableSchema();
+                    }
+                    return AppendClientInfo.of(
+                            tableSchema,
+                            // Make sure that the client is always closed in a different thread to
+                            // avoid
+                            // blocking.
+                            client -> runAsyncIgnoreFailure(closeWriterExecutor, client::close))
+                        .withAppendClient(datasetService, getOrCreateStream, false);
+                  }));
+      if (autoUpdateSchema && updatedSchema.read() != null) {
+        if (appendClientInfo.get().hasSchemaChanged(updatedSchema.read())) {
+          appendClientInfo.set(
+              AppendClientInfo.of(
+                  updatedSchema.read(), appendClientInfo.get().getCloseAppendClient()));
+          // TODO: invalidate?

Review Comment:
   it seems that the unsharded version of the writes do invalidate when schema has changed. 



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java:
##########
@@ -488,6 +542,14 @@ long flush(
               return RetryType.RETRY_ALL_OPERATIONS;
             },
             c -> {
+              AppendRowsResponse response = Preconditions.checkStateNotNull(c.getResult());
+              if (autoUpdateSchema && response.hasUpdatedSchema()) {

Review Comment:
   do we get a similar information piece on schema updates when the insert fails? if so that info added to the failed insert rows could help on debugging. 



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -640,6 +678,18 @@ public void process(
         }
         appendSplitDistribution.update(numAppends);
 
+        if (updatedSchemaReturned.get() != null) {
+          // Update the table schema and clear the append client.
+          if (appendClientInfo.get().hasSchemaChanged(updatedSchemaReturned.get())) {
+            appendClientInfo.set(
+                AppendClientInfo.of(
+                    updatedSchemaReturned.get(), appendClientInfo.get().getCloseAppendClient()));
+            // TODO: invalidate?

Review Comment:
   same as in line 434



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java:
##########
@@ -61,9 +61,15 @@ public TableSchema getTableSchema() {
     }
 
     @Override
+    @SuppressWarnings("nullness")
     public StorageApiWritePayload toMessage(T element) {
       Message msg = BeamRowToStorageApiProto.messageFromBeamRow(descriptor, toRow.apply(element));
-      return new AutoValue_StorageApiWritePayload(msg.toByteArray());
+      return new AutoValue_StorageApiWritePayload(msg.toByteArray(), null);
+    }
+
+    @Override
+    public StorageApiWritePayload toMessage(TableRow tableRow, boolean respectRequired) {
+      throw new RuntimeException("Not supported");

Review Comment:
   Maybe I misunderstood the logic, but would it be possible to add a validation in BigQueryIO that schema update is not available for Beam Row payloads? 
   If I'm not mistaken someone could configure the writes with `useBeamSchema()` and set also auto update of schemas and then their pipeline will fail with this runtime exception, when in fact we could have captured that in validation time. 
   



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java:
##########
@@ -355,9 +385,31 @@ void invalidateWriteStream() {
         }
       }
 
-      void addMessage(StorageApiWritePayload payload) throws Exception {
+      void addMessage(
+          StorageApiWritePayload payload,
+          OutputReceiver<BigQueryStorageApiInsertError> failedRowsReceiver)
+          throws Exception {
         maybeTickleCache();
         ByteString payloadBytes = ByteString.copyFrom(payload.getPayload());
+        if (autoUpdateSchema) {
+          if (appendClientInfo == null) {
+            appendClientInfo = getAppendClientInfo(true, null);
+          }
+          @Nullable TableRow unknownFields = payload.getUnknownFields();
+          if (unknownFields != null) {
+            try {
+              payloadBytes =
+                  payloadBytes.concat(
+                      Preconditions.checkStateNotNull(appendClientInfo)
+                          .encodeUnknownFields(unknownFields, ignoreUnknownValues));
+            } catch (TableRowToStorageApiProto.SchemaConversionException e) {
+              TableRow tableRow = appendClientInfo.toTableRow(payloadBytes);
+              // TODO(reuvenlax): We need to merge the unknown fields in!

Review Comment:
   maybe just include the information of the unknown fields to the error message returned would be sufficient for debugging purposes. 



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org