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 2020/12/07 21:53:28 UTC

[GitHub] [beam] nehsyc commented on a change in pull request #13496: [BEAM-11408] Integrate BigQuery sink streaming inserts with GroupIntoBatches

nehsyc commented on a change in pull request #13496:
URL: https://github.com/apache/beam/pull/13496#discussion_r537860362



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.SinkMetrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupIntoBatches;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.Histogram;
+import org.apache.beam.sdk.util.ShardedKey;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+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.math.DoubleMath;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** PTransform to perform batched streaming BigQuery write. */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+class BatchedStreamingWrite<ErrorT, ElementT>
+    extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollection<ErrorT>> {
+  private static final TupleTag<Void> mainOutputTag = new TupleTag<>("mainOutput");
+
+  private final BigQueryServices bqServices;
+  private final InsertRetryPolicy retryPolicy;
+  private final TupleTag<ErrorT> failedOutputTag;
+  private final AtomicCoder<ErrorT> failedOutputCoder;
+  private final ErrorContainer<ErrorT> errorContainer;
+  private final boolean skipInvalidRows;
+  private final boolean ignoreUnknownValues;
+  private final boolean ignoreInsertIds;
+  private final SerializableFunction<ElementT, TableRow> toTableRow;
+  private final SerializableFunction<ElementT, TableRow> toFailsafeTableRow;
+
+  /** Tracks histogram of bytes written. Reset at the start of every bundle. */
+  private transient Histogram histogram = Histogram.linear(0, 20, 3000);
+
+  private transient Long lastReportedSystemClockMillis = System.currentTimeMillis();
+
+  private final Logger LOG = LoggerFactory.getLogger(BatchedStreamingWrite.class);
+
+  /** Tracks bytes written, exposed as "ByteCount" Counter. */
+  private Counter byteCounter = SinkMetrics.bytesWritten();
+
+  /** Switches the method of batching. */
+  private final boolean batchViaStateful;
+
+  public BatchedStreamingWrite(
+      BigQueryServices bqServices,
+      InsertRetryPolicy retryPolicy,
+      TupleTag<ErrorT> failedOutputTag,
+      AtomicCoder<ErrorT> failedOutputCoder,
+      ErrorContainer<ErrorT> errorContainer,
+      boolean skipInvalidRows,
+      boolean ignoreUnknownValues,
+      boolean ignoreInsertIds,
+      SerializableFunction<ElementT, TableRow> toTableRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow) {
+    this.bqServices = bqServices;
+    this.retryPolicy = retryPolicy;
+    this.failedOutputTag = failedOutputTag;
+    this.failedOutputCoder = failedOutputCoder;
+    this.errorContainer = errorContainer;
+    this.skipInvalidRows = skipInvalidRows;
+    this.ignoreUnknownValues = ignoreUnknownValues;
+    this.ignoreInsertIds = ignoreInsertIds;
+    this.toTableRow = toTableRow;
+    this.toFailsafeTableRow = toFailsafeTableRow;
+    this.batchViaStateful = false;
+  }
+
+  private BatchedStreamingWrite(
+      BigQueryServices bqServices,
+      InsertRetryPolicy retryPolicy,
+      TupleTag<ErrorT> failedOutputTag,
+      AtomicCoder<ErrorT> failedOutputCoder,
+      ErrorContainer<ErrorT> errorContainer,
+      boolean skipInvalidRows,
+      boolean ignoreUnknownValues,
+      boolean ignoreInsertIds,
+      SerializableFunction<ElementT, TableRow> toTableRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow,
+      boolean batchViaStateful) {
+    this.bqServices = bqServices;
+    this.retryPolicy = retryPolicy;
+    this.failedOutputTag = failedOutputTag;
+    this.failedOutputCoder = failedOutputCoder;
+    this.errorContainer = errorContainer;
+    this.skipInvalidRows = skipInvalidRows;
+    this.ignoreUnknownValues = ignoreUnknownValues;
+    this.ignoreInsertIds = ignoreInsertIds;
+    this.toTableRow = toTableRow;
+    this.toFailsafeTableRow = toFailsafeTableRow;
+    this.batchViaStateful = batchViaStateful;
+  }
+
+  /**
+   * A transform that performs batched streaming BigQuery write; input elements are batched and
+   * flushed upon bundle finalization.
+   */
+  public BatchedStreamingWrite<ErrorT, ElementT> viaDoFnFinalization() {
+    return new BatchedStreamingWrite<>(
+        bqServices,
+        retryPolicy,
+        failedOutputTag,
+        failedOutputCoder,
+        errorContainer,
+        skipInvalidRows,
+        ignoreUnknownValues,
+        ignoreInsertIds,
+        toTableRow,
+        toFailsafeTableRow,
+        false);
+  }
+
+  /**
+   * A transform that performs batched streaming BigQuery write; input elements are grouped on table
+   * destinations and batched via a stateful DoFn. This also enables dynamic sharding during
+   * grouping to parallelize writes.
+   */
+  public BatchedStreamingWrite<ErrorT, ElementT> viaStateful() {
+    return new BatchedStreamingWrite<>(
+        bqServices,
+        retryPolicy,
+        failedOutputTag,
+        failedOutputCoder,
+        errorContainer,
+        skipInvalidRows,
+        ignoreUnknownValues,
+        ignoreInsertIds,
+        toTableRow,
+        toFailsafeTableRow,
+        true);
+  }
+
+  @Override
+  public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+    return batchViaStateful
+        ? input.apply(new ViaStateful())
+        : input.apply(new ViaBundleFinalization());
+  }
+
+  private class ViaBundleFinalization
+      extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollection<ErrorT>> {
+    @Override
+    public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+      PCollectionTuple result =
+          input.apply(
+              ParDo.of(new BatchAndInsertElements())
+                  .withOutputTags(mainOutputTag, TupleTagList.of(failedOutputTag)));
+      PCollection<ErrorT> failedInserts = result.get(failedOutputTag);
+      failedInserts.setCoder(failedOutputCoder);
+      return failedInserts;
+    }
+  }
+
+  @VisibleForTesting
+  private class BatchAndInsertElements extends DoFn<KV<String, TableRowInfo<ElementT>>, Void> {
+
+    /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */
+    private transient Map<String, List<FailsafeValueInSingleWindow<TableRow, TableRow>>> tableRows;
+
+    /** The list of unique ids for each BigQuery table row. */
+    private transient Map<String, List<String>> uniqueIdsForTableRows;
+
+    @Setup
+    public void setup() {
+      // record latency upto 60 seconds in the resolution of 20ms
+      histogram = Histogram.linear(0, 20, 3000);
+      lastReportedSystemClockMillis = System.currentTimeMillis();
+    }
+
+    @Teardown
+    public void teardown() {
+      if (histogram.getTotalCount() > 0) {
+        logPercentiles();
+        histogram.clear();
+      }
+    }
+
+    /** Prepares a target BigQuery table. */
+    @StartBundle
+    public void startBundle() {
+      tableRows = new HashMap<>();
+      uniqueIdsForTableRows = new HashMap<>();
+    }
+
+    /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */
+    @ProcessElement
+    public void processElement(
+        @Element KV<String, TableRowInfo<ElementT>> element,
+        @Timestamp Instant timestamp,
+        BoundedWindow window,
+        PaneInfo pane) {
+      String tableSpec = element.getKey();
+      List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
+          BigQueryHelpers.getOrCreateMapListValue(tableRows, tableSpec);
+      List<String> uniqueIds =
+          BigQueryHelpers.getOrCreateMapListValue(uniqueIdsForTableRows, tableSpec);
+
+      TableRow tableRow = toTableRow.apply(element.getValue().tableRow);
+      TableRow failsafeTableRow = toFailsafeTableRow.apply(element.getValue().tableRow);
+      rows.add(FailsafeValueInSingleWindow.of(tableRow, timestamp, window, pane, failsafeTableRow));
+      uniqueIds.add(element.getValue().uniqueId);
+    }
+
+    /** Writes the accumulated rows into BigQuery with streaming API. */
+    @FinishBundle
+    public void finishBundle(FinishBundleContext context) throws Exception {
+      List<ValueInSingleWindow<ErrorT>> failedInserts = Lists.newArrayList();
+      BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class);
+      for (Map.Entry<String, List<FailsafeValueInSingleWindow<TableRow, TableRow>>> entry :
+          tableRows.entrySet()) {
+        TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey());
+        flushRows(
+            tableReference,
+            entry.getValue(),
+            uniqueIdsForTableRows.get(entry.getKey()),
+            options,
+            failedInserts);
+      }
+      tableRows.clear();
+      uniqueIdsForTableRows.clear();
+
+      for (ValueInSingleWindow<ErrorT> row : failedInserts) {
+        context.output(failedOutputTag, row.getValue(), row.getTimestamp(), row.getWindow());
+      }
+
+      updateAndLogHistogram(options);
+    }
+  }
+
+  private class ViaStateful
+      extends PTransform<PCollection<KV<String, TableRowInfo<ElementT>>>, PCollection<ErrorT>> {
+    private final Duration BATCH_MAX_BUFFERING_DURATION = Duration.standardSeconds(10);
+
+    @Override
+    public PCollection<ErrorT> expand(PCollection<KV<String, TableRowInfo<ElementT>>> input) {
+      BigQueryOptions options = input.getPipeline().getOptions().as(BigQueryOptions.class);
+      KvCoder<String, TableRowInfo<ElementT>> inputCoder = (KvCoder) input.getCoder();
+      TableRowInfoCoder<ElementT> valueCoder =
+          (TableRowInfoCoder) inputCoder.getCoderArguments().get(1);
+      PCollectionTuple result =
+          input

Review comment:
       Didn't add a global window around here as the existing implementation:
   https://github.com/apache/beam/blob/2462fe996626874a6a11f35c00405acc831c8dfd/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java#L281
   
   According to the documentation the global window is for correct access to side inputs but it seems to me that up to this point the dynamic destination has been read. @reuvenlax I might be missing something but let me know if I should keep the global window (and if so where?).




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