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/11/02 21:51:45 UTC

[GitHub] [beam] pabloem commented on a change in pull request #15810: [BEAM-2791] Support low-latency StorageApi sink with no exactly-once guarantees

pabloem commented on a change in pull request #15810:
URL: https://github.com/apache/beam/pull/15810#discussion_r741463763



##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java
##########
@@ -85,6 +85,13 @@
 
   void setUseStorageWriteApi(Boolean value);
 
+  @Description(
+      "If set, then BigQueryIO.Write will default to using the approximate Storage Write API.")

Review comment:
       ```suggestion
         "If set, then BigQueryIO.Write will use the Storage Write API with lower latency and relaxed consistency guarantees.")
   ```
   Something like that?

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse;
+import com.google.cloud.bigquery.storage.v1beta2.ProtoRows;
+import com.google.protobuf.Descriptors.Descriptor;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient;
+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.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+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.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.RemovalNotification;
+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;
+
+/**
+ * A transform to write sharded records to BigQuery using the Storage API. This transform uses the
+ * default stream to write the records. Records written will show up in BigQuery immediately,
+ * however exactly once is not guaranteed - duplicates may appear in the output. For exactly-once
+ * writes, use {@link StorageApiWritesShardedRecords} or {@link StorageApiWriteUnshardedRecords}.
+ */
+@SuppressWarnings("FutureReturnValueIgnored")
+public class StorageApiWriteRecordsInconsistent<DestinationT, ElementT>
+    extends PTransform<PCollection<KV<DestinationT, byte[]>>, PCollection<Void>> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageApiWriteRecordsInconsistent.class);
+
+  private final StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations;
+  private final CreateDisposition createDisposition;
+  private final String kmsKey;
+  private final BigQueryServices bqServices;
+  private final Coder<DestinationT> destinationCoder;
+  private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool();
+
+  private static final Cache<String, StreamAppendClient> APPEND_CLIENTS =
+      CacheBuilder.newBuilder()
+          .expireAfterAccess(5, TimeUnit.MINUTES)
+          .removalListener(
+              (RemovalNotification<String, StreamAppendClient> removal) -> {
+                @Nullable final StreamAppendClient streamAppendClient = removal.getValue();
+                // Close the writer in a different thread so as not to block the main one.
+                runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close);
+              })
+          .build();
+
+  // Run a closure asynchronously, ignoring failures.
+  private interface ThrowingRunnable {
+    void run() throws Exception;
+  }
+
+  private static void runAsyncIgnoreFailure(ExecutorService executor, ThrowingRunnable task) {
+    executor.submit(
+        () -> {
+          try {
+            task.run();
+          } catch (Exception e) {
+            //
+          }
+        });
+  }
+
+  public StorageApiWriteRecordsInconsistent(
+      StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations,
+      CreateDisposition createDisposition,
+      String kmsKey,
+      BigQueryServices bqServices,
+      Coder<DestinationT> destinationCoder) {
+    this.dynamicDestinations = dynamicDestinations;
+    this.createDisposition = createDisposition;
+    this.kmsKey = kmsKey;
+    this.bqServices = bqServices;
+    this.destinationCoder = destinationCoder;
+  }
+
+  @Override
+  public PCollection<Void> expand(PCollection<KV<DestinationT, byte[]>> input) {
+    String operationName = input.getName() + "/" + getName();
+    // Append records to the Storage API streams.
+    return input.apply(
+        "Write Records",
+        ParDo.of(new WriteRecordsDoFn(operationName))
+            .withSideInputs(dynamicDestinations.getSideInputs()));
+  }
+
+  class WriteRecordsDoFn extends DoFn<KV<DestinationT, byte[]>, Void> {
+    private final TwoLevelMessageConverterCache<DestinationT, ElementT> messageConverters;
+    private Map<DestinationT, TableDestination> destinations = Maps.newHashMap();
+    private Map<DestinationT, List<byte[]>> bundleElements = Maps.newHashMap();
+    private @Nullable DatasetService datasetServiceInternal = null;
+    private final Counter recordsAppended =
+        Metrics.counter(WriteRecordsDoFn.class, "recordsAppended");
+    private final Counter appendFailures =
+        Metrics.counter(WriteRecordsDoFn.class, "appendFailures");
+    private final Distribution appendLatencyDistribution =
+        Metrics.distribution(WriteRecordsDoFn.class, "appendLatencyDistributionMs");
+
+    public WriteRecordsDoFn(String operationName) {
+      this.messageConverters = new TwoLevelMessageConverterCache<>(operationName);
+    }
+
+    @StartBundle
+    public void startBundle() {
+      destinations = Maps.newHashMap();
+      bundleElements = Maps.newHashMap();
+    }
+
+    @ProcessElement
+    public void process(ProcessContext c, @Element KV<DestinationT, byte[]> element)
+        throws Exception {
+      dynamicDestinations.setSideInputAccessorFromProcessContext(c);
+      destinations.computeIfAbsent(
+          element.getKey(),
+          dest -> {
+            TableDestination tableDestination1 = dynamicDestinations.getTable(dest);
+            checkArgument(
+                tableDestination1 != null,
+                "DynamicDestinations.getTable() may not return null, "
+                    + "but %s returned null for destination %s",
+                dynamicDestinations,
+                dest);
+            Supplier<TableSchema> schemaSupplier = () -> dynamicDestinations.getSchema(dest);
+            return CreateTableHelpers.possiblyCreateTable(
+                c,
+                tableDestination1,
+                schemaSupplier,
+                createDisposition,
+                destinationCoder,
+                kmsKey,
+                bqServices);
+          });
+      bundleElements
+          .computeIfAbsent(element.getKey(), dest -> Lists.newArrayList())
+          .add(element.getValue());
+    }
+
+    @FinishBundle
+    public void finishBundle(PipelineOptions pipelineOptions) throws Exception {
+      final DatasetService datasetService = getDatasetService(pipelineOptions);
+      Instant now = Instant.now();
+      RetryManager<AppendRowsResponse, RetryManager.Operation.Context<AppendRowsResponse>>
+          retryManager =
+              new RetryManager<>(Duration.standardSeconds(1), Duration.standardSeconds(10), 1000);
+      for (Map.Entry<DestinationT, List<byte[]>> entry : bundleElements.entrySet()) {
+        appendRecords(entry.getKey(), entry.getValue(), datasetService, retryManager);
+      }
+      retryManager.run(true);
+
+      java.time.Duration timeElapsed = java.time.Duration.between(now, Instant.now());
+      appendLatencyDistribution.update(timeElapsed.toMillis());
+
+      destinations.clear();
+      bundleElements.clear();
+    }
+
+    @SuppressWarnings({"nullness"})
+    public void appendRecords(
+        DestinationT destination,
+        List<byte[]> records,
+        DatasetService datasetService,
+        RetryManager<AppendRowsResponse, RetryManager.Operation.Context<AppendRowsResponse>>
+            retryManager)
+        throws Exception {
+      @Nullable TableDestination tableDestination = destinations.get(destination);
+      if (tableDestination == null) {
+        throw new RuntimeException("Unexpected null destination.");
+      }
+      final String tableId = tableDestination.getTableUrn();
+      MessageConverter<ElementT> messageConverter =
+          messageConverters.get(destination, dynamicDestinations);
+      Descriptor descriptor = messageConverter.getSchemaDescriptor();
+      final long oneMb = 1024 * 1024;

Review comment:
       perhaps make this a constant with documentation?

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse;
+import com.google.cloud.bigquery.storage.v1beta2.ProtoRows;
+import com.google.protobuf.Descriptors.Descriptor;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient;
+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.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+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.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.RemovalNotification;
+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;
+
+/**
+ * A transform to write sharded records to BigQuery using the Storage API. This transform uses the
+ * default stream to write the records. Records written will show up in BigQuery immediately,
+ * however exactly once is not guaranteed - duplicates may appear in the output. For exactly-once
+ * writes, use {@link StorageApiWritesShardedRecords} or {@link StorageApiWriteUnshardedRecords}.
+ */
+@SuppressWarnings("FutureReturnValueIgnored")
+public class StorageApiWriteRecordsInconsistent<DestinationT, ElementT>
+    extends PTransform<PCollection<KV<DestinationT, byte[]>>, PCollection<Void>> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageApiWriteRecordsInconsistent.class);
+
+  private final StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations;
+  private final CreateDisposition createDisposition;
+  private final String kmsKey;
+  private final BigQueryServices bqServices;
+  private final Coder<DestinationT> destinationCoder;
+  private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool();
+
+  private static final Cache<String, StreamAppendClient> APPEND_CLIENTS =
+      CacheBuilder.newBuilder()
+          .expireAfterAccess(5, TimeUnit.MINUTES)
+          .removalListener(
+              (RemovalNotification<String, StreamAppendClient> removal) -> {
+                @Nullable final StreamAppendClient streamAppendClient = removal.getValue();
+                // Close the writer in a different thread so as not to block the main one.
+                runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close);
+              })
+          .build();
+
+  // Run a closure asynchronously, ignoring failures.
+  private interface ThrowingRunnable {
+    void run() throws Exception;
+  }
+
+  private static void runAsyncIgnoreFailure(ExecutorService executor, ThrowingRunnable task) {
+    executor.submit(
+        () -> {
+          try {
+            task.run();
+          } catch (Exception e) {
+            //
+          }
+        });
+  }
+
+  public StorageApiWriteRecordsInconsistent(
+      StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations,
+      CreateDisposition createDisposition,
+      String kmsKey,
+      BigQueryServices bqServices,
+      Coder<DestinationT> destinationCoder) {
+    this.dynamicDestinations = dynamicDestinations;
+    this.createDisposition = createDisposition;
+    this.kmsKey = kmsKey;
+    this.bqServices = bqServices;
+    this.destinationCoder = destinationCoder;
+  }
+
+  @Override
+  public PCollection<Void> expand(PCollection<KV<DestinationT, byte[]>> input) {
+    String operationName = input.getName() + "/" + getName();
+    // Append records to the Storage API streams.
+    return input.apply(
+        "Write Records",
+        ParDo.of(new WriteRecordsDoFn(operationName))

Review comment:
       is the difference between the inconsistent and consistent writes the finalization of streams? (e.g. like here: https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java#L103)
   
   Doesn't it make sense to reuse the logic from that file? I see this file has metrics, so that's nice... just wondering. WDYT?

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteRecordsInconsistent.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse;
+import com.google.cloud.bigquery.storage.v1beta2.ProtoRows;
+import com.google.protobuf.Descriptors.Descriptor;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient;
+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.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+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.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.RemovalNotification;
+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;
+
+/**
+ * A transform to write sharded records to BigQuery using the Storage API. This transform uses the
+ * default stream to write the records. Records written will show up in BigQuery immediately,
+ * however exactly once is not guaranteed - duplicates may appear in the output. For exactly-once
+ * writes, use {@link StorageApiWritesShardedRecords} or {@link StorageApiWriteUnshardedRecords}.
+ */
+@SuppressWarnings("FutureReturnValueIgnored")
+public class StorageApiWriteRecordsInconsistent<DestinationT, ElementT>
+    extends PTransform<PCollection<KV<DestinationT, byte[]>>, PCollection<Void>> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageApiWriteRecordsInconsistent.class);
+
+  private final StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations;
+  private final CreateDisposition createDisposition;
+  private final String kmsKey;
+  private final BigQueryServices bqServices;
+  private final Coder<DestinationT> destinationCoder;
+  private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool();
+
+  private static final Cache<String, StreamAppendClient> APPEND_CLIENTS =
+      CacheBuilder.newBuilder()
+          .expireAfterAccess(5, TimeUnit.MINUTES)
+          .removalListener(
+              (RemovalNotification<String, StreamAppendClient> removal) -> {
+                @Nullable final StreamAppendClient streamAppendClient = removal.getValue();
+                // Close the writer in a different thread so as not to block the main one.
+                runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close);
+              })
+          .build();
+
+  // Run a closure asynchronously, ignoring failures.
+  private interface ThrowingRunnable {
+    void run() throws Exception;
+  }
+
+  private static void runAsyncIgnoreFailure(ExecutorService executor, ThrowingRunnable task) {
+    executor.submit(
+        () -> {
+          try {
+            task.run();
+          } catch (Exception e) {
+            //
+          }
+        });
+  }
+
+  public StorageApiWriteRecordsInconsistent(
+      StorageApiDynamicDestinations<ElementT, DestinationT> dynamicDestinations,
+      CreateDisposition createDisposition,
+      String kmsKey,
+      BigQueryServices bqServices,
+      Coder<DestinationT> destinationCoder) {
+    this.dynamicDestinations = dynamicDestinations;
+    this.createDisposition = createDisposition;
+    this.kmsKey = kmsKey;
+    this.bqServices = bqServices;
+    this.destinationCoder = destinationCoder;
+  }
+
+  @Override
+  public PCollection<Void> expand(PCollection<KV<DestinationT, byte[]>> input) {
+    String operationName = input.getName() + "/" + getName();
+    // Append records to the Storage API streams.
+    return input.apply(

Review comment:
       Does it make sense to add `GroupIntoBatches.withShardedKey` in this transform? At least for streaming pipelines?




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