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 2021/03/07 09:29:10 UTC

[GitHub] [beam] yirutang commented on a change in pull request #11767: [BEAM-11648] BigQuery Storage API sink

yirutang commented on a change in pull request #11767:
URL: https://github.com/apache/beam/pull/11767#discussion_r589000464



##########
File path: examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java
##########
@@ -0,0 +1,133 @@
+/*

Review comment:
       High level, do you want to offer user an option of not to use deduplication? If user don't need deduplication, they could use default stream and subject to less quota and higher performance on writes. It is in COMMITTED mode only and more like using the old insertall without insert_id.

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.cloud.bigquery.storage.v1beta2.BatchCommitWriteStreamsResponse;
+import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamResponse;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
+import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context;
+import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+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.collect.Maps;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** This DoFn finalizes and commits Storage API streams. */
+class StorageApiFinalizeWritesDoFn extends DoFn<KV<String, String>, Void> {
+  private static final Logger LOG = LoggerFactory.getLogger(StorageApiFinalizeWritesDoFn.class);
+
+  private final Counter finalizeOperationsSent =
+      Metrics.counter(StorageApiFinalizeWritesDoFn.class, "finalizeOperationsSent");
+  private final Counter finalizeOperationsSucceeded =
+      Metrics.counter(StorageApiFinalizeWritesDoFn.class, "finalizeOperationsSucceeded");
+  private final Counter finalizeOperationsFailed =
+      Metrics.counter(StorageApiFinalizeWritesDoFn.class, "finalizeOperationsFailed");
+  private final Counter batchCommitOperationsSent =
+      Metrics.counter(StorageApiFinalizeWritesDoFn.class, "batchCommitOperationsSent");
+  private final Counter batchCommitOperationsSucceeded =
+      Metrics.counter(StorageApiFinalizeWritesDoFn.class, "batchCommitOperationsSucceeded");
+  private final Counter batchCommitOperationsFailed =
+      Metrics.counter(StorageApiFinalizeWritesDoFn.class, "batchCommitOperationsFailed");
+
+  private Map<String, Collection<String>> commitStreams;
+  private final BigQueryServices bqServices;
+  @Nullable private DatasetService datasetService;
+
+  public StorageApiFinalizeWritesDoFn(BigQueryServices bqServices) {
+    this.bqServices = bqServices;
+    this.commitStreams = Maps.newHashMap();
+    this.datasetService = null;
+  }
+
+  private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException {
+    if (datasetService == null) {
+      datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class));
+    }
+    return datasetService;
+  }
+
+  @StartBundle
+  public void startBundle() throws IOException {
+    commitStreams = Maps.newHashMap();
+  }
+
+  @ProcessElement
+  @SuppressWarnings({"nullness"})
+  public void process(PipelineOptions pipelineOptions, @Element KV<String, String> element)
+      throws Exception {
+    String tableId = element.getKey();
+    String streamId = element.getValue();
+    DatasetService datasetService = getDatasetService(pipelineOptions);
+
+    RetryManager<FinalizeWriteStreamResponse, Context<FinalizeWriteStreamResponse>> retryManager =
+        new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3);
+    retryManager.addOperation(
+        c -> {
+          finalizeOperationsSent.inc();
+          return datasetService.finalizeWriteStream(streamId);
+        },
+        contexts -> {
+          LOG.error(
+              "Finalize of stream "
+                  + streamId
+                  + " failed with "
+                  + Iterables.getFirst(contexts, null).getError());
+          finalizeOperationsFailed.inc();
+          return RetryType.RETRY_ALL_OPERATIONS;
+        },
+        c -> {
+          LOG.info("Finalize of stream " + streamId + " finished with " + c.getResult());
+          finalizeOperationsSucceeded.inc();
+          commitStreams.computeIfAbsent(tableId, d -> Lists.newArrayList()).add(streamId);
+        },
+        new Context<>());
+    retryManager.run(true);
+  }
+
+  @FinishBundle
+  @SuppressWarnings({"nullness"})
+  public void finishBundle(PipelineOptions pipelineOptions) throws Exception {
+    DatasetService datasetService = getDatasetService(pipelineOptions);
+    for (Map.Entry<String, Collection<String>> entry : commitStreams.entrySet()) {
+      final String tableId = entry.getKey();
+      final Collection<String> streamNames = entry.getValue();
+      RetryManager<BatchCommitWriteStreamsResponse, Context<BatchCommitWriteStreamsResponse>>
+          retryManager =
+              new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3);
+      retryManager.addOperation(
+          c -> {
+            batchCommitOperationsSent.inc();

Review comment:
       FYI, BatchCommit will return detailed error info. Say if it thinks in the batch, some streams are already committed, then it will return the list of already committed streams here:
   https://source.corp.google.com/piper///depot/google3/google/cloud/bigquery/storage/v1beta2/storage.proto;l=751?q=BatchCommitWriteStreams
   
   To see if the operation really succeeded, commit_time in the response cannot be empty.




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

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