[
https://issues.apache.org/jira/browse/BEAM-2791?focusedWorklogId=674325&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-674325
]
ASF GitHub Bot logged work on BEAM-2791:
----------------------------------------
Author: ASF GitHub Bot
Created on: 03/Nov/21 02:32
Start Date: 03/Nov/21 02:32
Worklog Time Spent: 10m
Work Description: reuvenlax commented on a change in pull request #15810:
URL: https://github.com/apache/beam/pull/15810#discussion_r741580524
##########
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:
StorageApiWriteUnshardedRecords implements a fairly intricate protocol
to ensure exactly-once writes to BQ. It involves shuffles to ensure consistent
mapping of records to streams, coordination of stream creation, appends, and
flushes, and a protocol to throw away and recreate streams to prevent
duplicates on certain errors.
This PR adds a much simpler transform that does not guarantee exactly once.
No shuffle is needed, and the default Storage API stream is used. None of the
flushing or stream creation/recreation is needed either. I don't think it would
be easy to use that other file, though there might be some code snippets in
both files that could be refactored into a helper class.
--
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 674325)
Time Spent: 2h 40m (was: 2.5h)
> DirectRunner shuts down when there are pending event time timers
> ----------------------------------------------------------------
>
> Key: BEAM-2791
> URL: https://issues.apache.org/jira/browse/BEAM-2791
> Project: Beam
> Issue Type: Bug
> Components: runner-direct
> Reporter: Kenneth Knowles
> Priority: P3
> Labels: sickbay
> Time Spent: 2h 40m
> Remaining Estimate: 0h
>
> When there are pending event time timers for a window by definition they are
> within the window so it should not be possible to GC the window nor to shut
> down the pipeline.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)