ahmedabu98 commented on code in PR #38264:
URL: https://github.com/apache/beam/pull/38264#discussion_r3154653273


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -487,6 +446,46 @@ public void onTeardown() {
       }
     }
 
+    // Holder for an AppendClientHolder. Maintains a pin on the client as long 
as it's active.
+    private class AppendClientHolder implements AutoCloseable {

Review Comment:
   Instead of pinning for each append context, we're using one 
AppendClientHolder that holds a pin on their behalf?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -282,7 +242,6 @@ static class AppendRowsContext<DestinationT>
       extends RetryManager.Operation.Context<AppendRowsResponse> {
     final ShardedKey<DestinationT> key;
     String streamName = "";
-    @Nullable StreamAppendClient client = null;

Review Comment:
   Hmmm so we're no longer pinning the client for each AppenRowsContext? Can 
you briefly explain this change somewhere? maybe in the PR description?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -671,475 +880,223 @@ public void process(
                 AppendClientInfo.of(
                         Preconditions.checkStateNotNull(tableSchema),
                         descriptor,
-                        // Make sure that the client is always closed in a 
different thread
-                        // to
-                        // avoid blocking.
-                        client ->
-                            runAsyncIgnoreFailure(
-                                closeWriterExecutor,
-                                () -> {
-                                  // Remove the pin that is "owned" by the 
cache.
-                                  client.unpin();
-                                  client.close();
-                                }))
+                        AutoCloseable::close)
                     .withAppendClient(
                         writeStreamService,
                         getOrCreateStream,
                         false,
                         defaultMissingValueInterpretation);
-            // This pin is "owned" by the cache.
-            
Preconditions.checkStateNotNull(info.getStreamAppendClient()).pin();
             return info;
           };
 
-      AtomicReference<AppendClientInfo> appendClientInfo =
-          new AtomicReference<>(
-              APPEND_CLIENTS.get(
-                  messageConverters.getAppendClientKey(element.getKey()), 
getAppendClientInfo));
-      String currentStream = getOrCreateStream.get();
-      if (!currentStream.equals(appendClientInfo.get().getStreamName())) {
-        // Cached append client is inconsistent with persisted state. Throw 
away cached item and
-        // force it to be
-        // recreated.
-        
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
-        appendClientInfo.set(
-            APPEND_CLIENTS.get(
-                messageConverters.getAppendClientKey(element.getKey()), 
getAppendClientInfo));
-      }
-
-      TableSchema updatedSchemaValue = updatedSchema.read();
-      if (autoUpdateSchema && updatedSchemaValue != null) {
-        if (appendClientInfo.get().hasSchemaChanged(updatedSchemaValue)) {
-          appendClientInfo.set(
-              AppendClientInfo.of(
-                  updatedSchemaValue, 
appendClientInfo.get().getCloseAppendClient(), false));
-          
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
-          APPEND_CLIENTS.put(
-              messageConverters.getAppendClientKey(element.getKey()), 
appendClientInfo.get());
+      // The StreamWriter has two pins on it. The static cache holds a pin, as 
it continues to cache
+      // values after this
+      // method exits, so must hold the pin. The local AppendClientHolder also 
holds a pin, as the
+      // cache could in
+      // theory evict the object during execution and we want a pin held 
throughout the execution of
+      // this function.
+      try (AppendClientHolder appendClientHolder =
+          new AppendClientHolder(element.getKey(), getAppendClientInfo)) {
+        String currentStream = getOrCreateStream.get();
+        if (!currentStream.equals(appendClientHolder.get().getStreamName())) {
+          // Cached append client is inconsistent with persisted state. Throw 
away cached item and
+          // force it to be recreated.
+          appendClientHolder.invalidateAndReset();
         }
-      }
 
-      // Initialize stream names and offsets for all contexts. This will be 
called initially, but
-      // will also be called if we roll over to a new stream on a retry.
-      BiConsumer<Iterable<AppendRowsContext<DestinationT>>, Boolean> 
initializeContexts =
-          (contexts, isFailure) -> {
-            try {
-              if (isFailure) {
-                // Clear the stream name, forcing a new one to be created.
-                streamName.write("");
-              }
-              appendClientInfo.set(
-                  appendClientInfo
-                      .get()
-                      .withAppendClient(
-                          writeStreamService,
-                          getOrCreateStream,
-                          false,
-                          defaultMissingValueInterpretation));
-              StreamAppendClient streamAppendClient =
-                  Preconditions.checkArgumentNotNull(
-                      appendClientInfo.get().getStreamAppendClient());
-              String streamNameRead = 
Preconditions.checkArgumentNotNull(streamName.read());
-              long currentOffset = 
Preconditions.checkArgumentNotNull(streamOffset.read());
-              for (AppendRowsContext<DestinationT> context : contexts) {
-                context.streamName = streamNameRead;
-                streamAppendClient.pin();
-                context.client = 
appendClientInfo.get().getStreamAppendClient();
-                context.offset = currentOffset;
-                ++context.tryIteration;
-                currentOffset = context.offset + 
context.protoRows.getSerializedRowsCount();
-              }
-              streamOffset.write(currentOffset);
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          };
-
-      Consumer<Iterable<AppendRowsContext<DestinationT>>> clearClients =
-          (contexts) -> {
-            
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
-            appendClientInfo.set(appendClientInfo.get().withNoAppendClient());
-            APPEND_CLIENTS.put(
-                messageConverters.getAppendClientKey(element.getKey()), 
appendClientInfo.get());
-            for (AppendRowsContext<DestinationT> context : contexts) {
-              if (context.client != null) {
-                // Unpin in a different thread, as it may execute a blocking 
close.
-                runAsyncIgnoreFailure(closeWriterExecutor, 
context.client::unpin);
-                context.client = null;
-              }
-            }
-          };
-
-      Function<AppendRowsContext<DestinationT>, ApiFuture<AppendRowsResponse>> 
runOperation =
-          context -> {
-            if (context.protoRows.getSerializedRowsCount() == 0) {
-              // This might happen if all rows in a batch failed and were sent 
to the failed-rows
-              // PCollection.
-              return 
ApiFutures.immediateFuture(AppendRowsResponse.newBuilder().build());
-            }
-            try {
-              appendClientInfo.set(
-                  appendClientInfo
-                      .get()
-                      .withAppendClient(
-                          writeStreamService,
-                          getOrCreateStream,
-                          false,
-                          defaultMissingValueInterpretation));
-              return 
Preconditions.checkStateNotNull(appendClientInfo.get().getStreamAppendClient())
-                  .appendRows(context.offset, context.protoRows);
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          };
+        TableSchema updatedSchemaValue = autoUpdateSchema ? 
updatedSchema.read() : null;
+        if (autoUpdateSchema && updatedSchemaValue != null) {
+          if (appendClientHolder.get().hasSchemaChanged(updatedSchemaValue)) {
+            appendClientHolder.invalidateAndReset();
+          }
+        }

Review Comment:
   nit: merge if conditions ?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java:
##########
@@ -134,7 +143,18 @@ public AppendClientInfo withAppendClient(
   public void close() {
     BigQueryServices.StreamAppendClient client = getStreamAppendClient();
     if (client != null) {
-      getCloseAppendClient().accept(client);
+      try {
+        getCloseAppendClient().accept(client);
+      } catch (Exception e) {
+        // We ignore errors when closing clients.
+        String msg =
+            e
+                + "\n"
+                + Arrays.stream(e.getStackTrace())
+                    .map(StackTraceElement::toString)
+                    .collect(Collectors.joining("\n"));
+        LOG.warn("Caught exception whilw trying to close append client. 
Ignoring {}", msg);

Review Comment:
   Gemini has a point here. Why not use `LOG.warn("Caught exception while 
trying to close append client. Ignoring", e);`?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientCache.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Encapsulates the cache of {@link AppendClientInfo} objects and the 
synchronization protocol
+ * required to use them safely. The Guava cache object is thread-safe. However 
our protocol requires
+ * that client pin the StreamAppendClient after looking up the cache, and we 
must ensure that the
+ * cache is not accessed in between the lookup and the pin (any access of the 
cache could trigger
+ * element expiration).
+ */
+class AppendClientCache<KeyT extends @NonNull Object> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(AppendClientCache.class);
+  private final ExecutorService closeWriterExecutor = 
Executors.newCachedThreadPool();
+
+  private final Cache<KeyT, AppendClientInfo> appendCache;
+
+  @SuppressWarnings({"FutureReturnValueIgnored"})
+  AppendClientCache(Duration expireAfterAccess) {
+    this.appendCache =
+        CacheBuilder.newBuilder()
+            .expireAfterAccess(expireAfterAccess.getMillis(), 
TimeUnit.MILLISECONDS)
+            .removalListener(
+                (RemovalNotification<KeyT, AppendClientInfo> removal) -> {
+                  LOG.info("Expiring append client for {}", removal.getKey());
+                  final @Nullable AppendClientInfo appendClientInfo = 
removal.getValue();
+                  if (appendClientInfo != null) {
+                    // Remove the pin owned by the cache itself. Since the 
client has not been
+                    // marked as closed, we
+                    // can call unpin in this thread without worrying about 
blocking the thread.
+                    appendClientInfo.unpinAppendClient(null);
+                    // Close the client in another thread to avoid blocking 
the main thread.
+                    closeWriterExecutor.submit(appendClientInfo::close);
+                  }
+                })
+            .build();
+  }
+
+  // The cache itself always own one pin on the object. This Callable is 
always used to ensure that
+  // the cache
+  // adds a pin before loading a value.
+  private static Callable<AppendClientInfo> 
wrapWithPin(Callable<AppendClientInfo> loader) {
+    return () -> {
+      AppendClientInfo client = loader.call();
+      client.pinAppendClient();
+      return client;
+    };
+  }
+
+  /**
+   * Atomically get an append client from the cache and add a pin. This pin is 
owned by the client,
+   * which has the responsibility of removing it. If the client is not in the 
cache, loader will be
+   * used to load the client; in this case an additional pin will be added 
owned by the cache,
+   * removed when the item is evicted.
+   */
+  public AppendClientInfo getAndPin(KeyT key, Callable<AppendClientInfo> 
loader) throws Exception {
+    synchronized (this) {
+      AppendClientInfo info = appendCache.get(key, wrapWithPin(loader));
+      info.pinAppendClient();
+      return info;
+    }
+  }
+
+  public AppendClientInfo putAndPin(KeyT key, Callable<AppendClientInfo> 
loader) throws Exception {
+    synchronized (this) {
+      AppendClientInfo info = wrapWithPin(loader).call();

Review Comment:
   Do we need to unpin any existing AppendClients?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -671,475 +880,223 @@ public void process(
                 AppendClientInfo.of(
                         Preconditions.checkStateNotNull(tableSchema),
                         descriptor,
-                        // Make sure that the client is always closed in a 
different thread
-                        // to
-                        // avoid blocking.
-                        client ->
-                            runAsyncIgnoreFailure(
-                                closeWriterExecutor,
-                                () -> {
-                                  // Remove the pin that is "owned" by the 
cache.
-                                  client.unpin();
-                                  client.close();
-                                }))
+                        AutoCloseable::close)
                     .withAppendClient(
                         writeStreamService,
                         getOrCreateStream,
                         false,
                         defaultMissingValueInterpretation);
-            // This pin is "owned" by the cache.
-            
Preconditions.checkStateNotNull(info.getStreamAppendClient()).pin();
             return info;
           };
 
-      AtomicReference<AppendClientInfo> appendClientInfo =
-          new AtomicReference<>(
-              APPEND_CLIENTS.get(
-                  messageConverters.getAppendClientKey(element.getKey()), 
getAppendClientInfo));
-      String currentStream = getOrCreateStream.get();
-      if (!currentStream.equals(appendClientInfo.get().getStreamName())) {
-        // Cached append client is inconsistent with persisted state. Throw 
away cached item and
-        // force it to be
-        // recreated.
-        
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
-        appendClientInfo.set(
-            APPEND_CLIENTS.get(
-                messageConverters.getAppendClientKey(element.getKey()), 
getAppendClientInfo));
-      }
-
-      TableSchema updatedSchemaValue = updatedSchema.read();
-      if (autoUpdateSchema && updatedSchemaValue != null) {
-        if (appendClientInfo.get().hasSchemaChanged(updatedSchemaValue)) {
-          appendClientInfo.set(
-              AppendClientInfo.of(
-                  updatedSchemaValue, 
appendClientInfo.get().getCloseAppendClient(), false));
-          
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
-          APPEND_CLIENTS.put(
-              messageConverters.getAppendClientKey(element.getKey()), 
appendClientInfo.get());
+      // The StreamWriter has two pins on it. The static cache holds a pin, as 
it continues to cache
+      // values after this
+      // method exits, so must hold the pin. The local AppendClientHolder also 
holds a pin, as the
+      // cache could in
+      // theory evict the object during execution and we want a pin held 
throughout the execution of
+      // this function.
+      try (AppendClientHolder appendClientHolder =
+          new AppendClientHolder(element.getKey(), getAppendClientInfo)) {
+        String currentStream = getOrCreateStream.get();
+        if (!currentStream.equals(appendClientHolder.get().getStreamName())) {
+          // Cached append client is inconsistent with persisted state. Throw 
away cached item and
+          // force it to be recreated.
+          appendClientHolder.invalidateAndReset();
         }
-      }
 
-      // Initialize stream names and offsets for all contexts. This will be 
called initially, but
-      // will also be called if we roll over to a new stream on a retry.
-      BiConsumer<Iterable<AppendRowsContext<DestinationT>>, Boolean> 
initializeContexts =
-          (contexts, isFailure) -> {
-            try {
-              if (isFailure) {
-                // Clear the stream name, forcing a new one to be created.
-                streamName.write("");
-              }
-              appendClientInfo.set(
-                  appendClientInfo
-                      .get()
-                      .withAppendClient(
-                          writeStreamService,
-                          getOrCreateStream,
-                          false,
-                          defaultMissingValueInterpretation));
-              StreamAppendClient streamAppendClient =
-                  Preconditions.checkArgumentNotNull(
-                      appendClientInfo.get().getStreamAppendClient());
-              String streamNameRead = 
Preconditions.checkArgumentNotNull(streamName.read());
-              long currentOffset = 
Preconditions.checkArgumentNotNull(streamOffset.read());
-              for (AppendRowsContext<DestinationT> context : contexts) {
-                context.streamName = streamNameRead;
-                streamAppendClient.pin();
-                context.client = 
appendClientInfo.get().getStreamAppendClient();
-                context.offset = currentOffset;
-                ++context.tryIteration;
-                currentOffset = context.offset + 
context.protoRows.getSerializedRowsCount();
-              }
-              streamOffset.write(currentOffset);
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          };
-
-      Consumer<Iterable<AppendRowsContext<DestinationT>>> clearClients =
-          (contexts) -> {
-            
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
-            appendClientInfo.set(appendClientInfo.get().withNoAppendClient());
-            APPEND_CLIENTS.put(
-                messageConverters.getAppendClientKey(element.getKey()), 
appendClientInfo.get());
-            for (AppendRowsContext<DestinationT> context : contexts) {
-              if (context.client != null) {
-                // Unpin in a different thread, as it may execute a blocking 
close.
-                runAsyncIgnoreFailure(closeWriterExecutor, 
context.client::unpin);
-                context.client = null;
-              }
-            }
-          };
-
-      Function<AppendRowsContext<DestinationT>, ApiFuture<AppendRowsResponse>> 
runOperation =
-          context -> {
-            if (context.protoRows.getSerializedRowsCount() == 0) {
-              // This might happen if all rows in a batch failed and were sent 
to the failed-rows
-              // PCollection.
-              return 
ApiFutures.immediateFuture(AppendRowsResponse.newBuilder().build());
-            }
-            try {
-              appendClientInfo.set(
-                  appendClientInfo
-                      .get()
-                      .withAppendClient(
-                          writeStreamService,
-                          getOrCreateStream,
-                          false,
-                          defaultMissingValueInterpretation));
-              return 
Preconditions.checkStateNotNull(appendClientInfo.get().getStreamAppendClient())
-                  .appendRows(context.offset, context.protoRows);
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          };
+        TableSchema updatedSchemaValue = autoUpdateSchema ? 
updatedSchema.read() : null;
+        if (autoUpdateSchema && updatedSchemaValue != null) {
+          if (appendClientHolder.get().hasSchemaChanged(updatedSchemaValue)) {
+            appendClientHolder.invalidateAndReset();
+          }
+        }
 
-      Function<Iterable<AppendRowsContext<DestinationT>>, RetryType> onError =
-          failedContexts -> {
-            // The first context is always the one that fails.
-            AppendRowsContext<DestinationT> failedContext =
-                
Preconditions.checkStateNotNull(Iterables.getFirst(failedContexts, null));
-            BigQuerySinkMetrics.reportFailedRPCMetrics(
-                failedContext, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, 
shortTableId);
-            String errorCode =
-                
BigQuerySinkMetrics.throwableToGRPCCodeString(failedContext.getError());
-
-            // AppendSerializationError means that BigQuery detected errors on 
individual rows, e.g.
-            // a row not conforming
-            // to bigQuery invariants. These errors are persistent, so we 
redirect those rows to the
-            // failedInserts
-            // PCollection, and retry with the remaining rows.
-            if (failedContext.getError() != null
-                && failedContext.getError() instanceof 
Exceptions.AppendSerializationError) {
-              Exceptions.AppendSerializationError error =
-                  Preconditions.checkArgumentNotNull(
-                      (Exceptions.AppendSerializationError) 
failedContext.getError());
-
-              Set<Integer> failedRowIndices = 
error.getRowIndexToErrorMessage().keySet();
-              for (int failedIndex : failedRowIndices) {
-                // Convert the message to a TableRow and send it to the 
failedRows collection.
-                TableRow failedRow = 
failedContext.failsafeTableRows.get(failedIndex);
-                if (failedRow == null) {
-                  // TODO: MAKE SURE WE USE UPDATED DESCRIPTOR
-                  ByteString protoBytes = 
failedContext.protoRows.getSerializedRows(failedIndex);
-                  failedRow =
-                      appendClientInfo.get().toTableRow(protoBytes, 
Predicates.alwaysTrue());
+        // Initialize stream names and offsets for all contexts. This will be 
called initially, but
+        // will also be called if we roll over to a new stream on a retry.
+        BiConsumer<Iterable<AppendRowsContext<DestinationT>>, Boolean> 
initializeContexts =
+            (contexts, isFailure) -> {
+              try {
+                if (isFailure) {
+                  // Clear the stream name, forcing a new one to be created.
+                  streamName.write("");
                 }
-                org.joda.time.Instant timestamp = 
failedContext.timestamps.get(failedIndex);
-                o.get(failedRowsTag)
-                    .outputWithTimestamp(
-                        new BigQueryStorageApiInsertError(
-                            failedRow,
-                            error.getRowIndexToErrorMessage().get(failedIndex),
-                            tableReference),
-                        timestamp);
-              }
-              int failedRows = failedRowIndices.size();
-              rowsSentToFailedRowsCollection.inc(failedRows);
-              BigQuerySinkMetrics.appendRowsRowStatusCounter(
-                      BigQuerySinkMetrics.RowStatus.FAILED, errorCode, 
shortTableId)
-                  .inc(failedRows);
-
-              // Remove the failed row from the payload, so we retry the batch 
without the failed
-              // rows.
-              ProtoRows.Builder retryRows = ProtoRows.newBuilder();
-              @Nullable List<org.joda.time.Instant> timestamps = 
Lists.newArrayList();
-              for (int i = 0; i < 
failedContext.protoRows.getSerializedRowsCount(); ++i) {
-                if (!failedRowIndices.contains(i)) {
-                  ByteString rowBytes = 
failedContext.protoRows.getSerializedRows(i);
-                  retryRows.addSerializedRows(rowBytes);
-                  timestamps.add(failedContext.timestamps.get(i));
+                String streamNameRead = 
Preconditions.checkArgumentNotNull(streamName.read());
+                long currentOffset = 
Preconditions.checkArgumentNotNull(streamOffset.read());
+                for (AppendRowsContext<DestinationT> context : contexts) {
+                  context.streamName = streamNameRead;
+                  context.offset = currentOffset;
+                  ++context.tryIteration;
+                  currentOffset = context.offset + 
context.protoRows.getSerializedRowsCount();
                 }
+                streamOffset.write(currentOffset);
+              } catch (Exception e) {
+                throw new RuntimeException(e);
               }
-              failedContext.protoRows = retryRows.build();
-              failedContext.timestamps = timestamps;
-              int retriedRows = 
failedContext.protoRows.getSerializedRowsCount();
-              BigQuerySinkMetrics.appendRowsRowStatusCounter(
-                      BigQuerySinkMetrics.RowStatus.RETRIED, errorCode, 
shortTableId)
-                  .inc(retriedRows);
-
-              // Since we removed rows, we need to update the insert offsets 
for all remaining rows.
-              long offset = failedContext.offset;
-              for (AppendRowsContext<DestinationT> context : failedContexts) {
-                context.offset = offset;
-                offset += context.protoRows.getSerializedRowsCount();
+            };
+
+        Consumer<Iterable<AppendRowsContext<DestinationT>>> clearClients =
+            (contexts) -> {
+              try {
+                appendClientHolder.invalidateAndReset();
+              } catch (Exception e) {
+                throw new RuntimeException(e);
               }
-              streamOffset.write(offset);
-              return RetryType.RETRY_ALL_OPERATIONS;
-            }
-
-            Throwable error = 
Preconditions.checkStateNotNull(failedContext.getError());
-
-            Status.Code statusCode = Status.fromThrowable(error).getCode();
-
-            // This means that the offset we have stored does not match the 
current end of
-            // the stream in the Storage API. Usually this happens because a 
crash or a bundle
-            // failure
-            // happened after an append but before the worker could checkpoint 
it's
-            // state. The records that were appended in a failed bundle will 
be retried,
-            // meaning that the unflushed tail of the stream must be discarded 
to prevent
-            // duplicates.
-            boolean offsetMismatch =
-                statusCode.equals(Code.OUT_OF_RANGE) || 
statusCode.equals(Code.ALREADY_EXISTS);
-
-            boolean quotaError = statusCode.equals(Code.RESOURCE_EXHAUSTED);
-            if (!offsetMismatch) {
-              // Don't log errors for expected offset mismatch. These will be 
logged as warnings
-              // below.
-              LOG.error(
-                  "Got error {} closing {}", failedContext.getError(), 
failedContext.streamName);
-            }
-
-            try {
-              // TODO: Only do this on explicit NOT_FOUND errors once BigQuery 
reliably produces
-              // them.
-              tryCreateTable.call();
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-
-            if (!quotaError) {
-              // For known errors (offset mismatch, not found) we must 
reestablish
-              // the streams.
-              // However we've seen that doing this fixes random stuckness 
issues by reestablishing
-              // gRPC connections,
-              // so we close the clients for all non-quota errors.
-
-              clearClients.accept(failedContexts);
-            }
-            appendFailures.inc();
-            int retriedRows = failedContext.protoRows.getSerializedRowsCount();
-            BigQuerySinkMetrics.appendRowsRowStatusCounter(
-                    BigQuerySinkMetrics.RowStatus.RETRIED, errorCode, 
shortTableId)
-                .inc(retriedRows);
-
-            // Schema mismatched exceptions can happen if the table was 
recently updated. Since
-            // vortex caches schemas
-            // we might see the new schema before vortex does. In this case, 
we simply need to
-            // retry.
-            Exceptions.@Nullable StorageException storageException =
-                Exceptions.toStorageException(error);
-            boolean schemaMismatchError =
-                (storageException instanceof 
Exceptions.SchemaMismatchedException);
-            if (!schemaMismatchError) {
-              // There's no special error code for missing required fields, 
and that can also happen
-              // due to vortex
-              // being delayed at seeing a new schema. We're forced to parse 
the description to
-              // determine that this has happened.
-              // TODO: Vortex team to introduce a special storage error code 
for this, so we don't
-              // have to parse
-              // descriptions.
-              Status status = Status.fromThrowable(error);
-              if (status.getCode() == Code.INVALID_ARGUMENT) {
-                String description = status.getDescription();
-                schemaMismatchError =
-                    description != null && description.contains("incompatible 
fields");
+            };
+
+        Function<AppendRowsContext<DestinationT>, 
ApiFuture<AppendRowsResponse>> runOperation =
+            context -> {
+              if (context.protoRows.getSerializedRowsCount() == 0) {
+                // This might happen if all rows in a batch failed and were 
sent to the failed-rows
+                // PCollection.
+                return 
ApiFutures.immediateFuture(AppendRowsResponse.newBuilder().build());
               }
-            }
-            if (schemaMismatchError) {
-              LOG.info(
-                  "Vortex failed stream open due to incompatible fields. This 
is likely because the BigTable "
-                      + "schema was recently updated and Vortex hasn't noticed 
yet, so retrying. error {}",
-                  Preconditions.checkStateNotNull(error).toString());
-            }
-
-            boolean explicitStreamFinalized =
-                failedContext.getError() instanceof StreamFinalizedException;
-            // This implies that the stream doesn't exist or has already been 
finalized. In this
-            // case we have no choice but to create a new stream.
-            boolean streamDoesNotExist =
-                explicitStreamFinalized
-                    || statusCode.equals(Code.INVALID_ARGUMENT)
-                    || statusCode.equals(Code.NOT_FOUND)
-                    || statusCode.equals(Code.FAILED_PRECONDITION);
-            streamDoesNotExist = streamDoesNotExist && !schemaMismatchError;
-
-            if (offsetMismatch || streamDoesNotExist) {
-              appendOffsetFailures.inc();
-              LOG.warn(
-                  "Append to {} failed. Will retry with a new stream",
-                  failedContext,
-                  failedContext.getError());
-              // Finalize the stream and clear streamName so a new stream will 
be created.
-              o.get(flushTag)
-                  .output(
-                      KV.of(
-                          failedContext.streamName, new 
Operation(failedContext.offset - 1, true)));
-              // Reinitialize all contexts with the new stream and new offsets.
-              initializeContexts.accept(failedContexts, true);
-
-              // Offset failures imply that all subsequent parallel appends 
will also fail.
-              // Retry them all.
+              try {
+                return appendClientHolder
+                    .getStreamAppendClient()
+                    .appendRows(context.offset, context.protoRows);
+              } catch (Exception e) {
+                throw new RuntimeException(e);
+              }
+            };
+
+        Function<Iterable<AppendRowsContext<DestinationT>>, RetryType> onError 
=
+            failedContexts -> {
+              handleAppendFailure(
+                  failedContexts,
+                  tableReference,
+                  shortTableId,
+                  appendClientHolder.get(),
+                  tryCreateTable,
+                  initializeContexts,
+                  clearClients,
+                  streamOffset,
+                  o);
               return RetryType.RETRY_ALL_OPERATIONS;
-            }
-
-            return RetryType.RETRY_ALL_OPERATIONS;
-          };
+            };
+        Consumer<AppendRowsContext<DestinationT>> onSuccess =
+            context -> handleAppendSuccess(context, shortTableId, 
appendClientHolder.get(), o);
+
+        BackOff backoff =
+            FluentBackoff.DEFAULT
+                .withInitialBackoff(Duration.standardSeconds(1))
+                .withMaxBackoff(Duration.standardMinutes(1))
+                .withMaxRetries(500)
+                .withThrottledTimeCounter(
+                    BigQuerySinkMetrics.throttledTimeCounter(
+                        BigQuerySinkMetrics.RpcMethod.OPEN_WRITE_STREAM))
+                .backoff();
+        CreateRetryManagerResult<DestinationT> createRetryManagerResult;
+        do {
+          // Each ProtoRows object contains at most 1MB of rows.
+          // TODO: Push messageFromTableRow up to top level. That we we cans 
skip TableRow entirely
+          // if
+          // already proto or already schema.
+          Iterable<SplittingIterable.Value> messages =
+              new SplittingIterable(
+                  element.getValue(),
+                  splitSize,
+                  // Unknown field merger
+                  (bytes, tableRow) ->
+                      appendClientHolder.get().mergeNewFields(bytes, tableRow, 
ignoreUnknownValues),
+                  // Convert back to TableRow
+                  bytes -> appendClientHolder.get().toTableRow(bytes, 
Predicates.alwaysTrue()),
+                  // Failed rows consumer
+                  (failedRow, errorMessage) -> {
+                    o.get(failedRowsTag)
+                        .outputWithTimestamp(
+                            new BigQueryStorageApiInsertError(
+                                failedRow.getValue(), errorMessage, 
tableReference),
+                            failedRow.getTimestamp());
+                    rowsSentToFailedRowsCollection.inc();
+                    BigQuerySinkMetrics.appendRowsRowStatusCounter(
+                            BigQuerySinkMetrics.RowStatus.FAILED,
+                            BigQuerySinkMetrics.PAYLOAD_TOO_LARGE,
+                            shortTableId)
+                        .inc(1);
+                  },
+                  // Get the currently-known TableSchema hash
+                  () -> appendClientHolder.get().getTableSchemaHash(),
+                  () ->
+                      TableRowToStorageApiProto.wrapDescriptorProto(
+                          messageConverter.getDescriptor(false)),
+                  autoUpdateSchema,
+                  elementTs);
+
+          createRetryManagerResult =
+              createRetryManager(
+                  element.getKey(),
+                  messages,
+                  runOperation,
+                  onError,
+                  onSuccess,
+                  appendClientHolder.get(),
+                  tableReference);
+          if (createRetryManagerResult.getSchemaMismatchSeen()) {
+            // TODO: The call to updateSchemaFromTable will throttle the DoFn 
(both because of the
+            // RPC
+            // call and because
+            // the cache has a delay on refresh). We should update throttling 
counters here as well.
+            LOG.info("Schema out of date: refreshing table schema for {}", 
tableId);
+            // Force the message converter to get the schema again from the 
table.
+            messageConverter.updateSchemaFromTable();
+            // Close all RPC clients that were opened with the old descriptor. 
Clear the cache,
+            // forcing us to create a new append client with the updated 
descriptor.
+            appendClientHolder.invalidateAndReset();
+          }
+        } while (createRetryManagerResult.getSchemaMismatchSeen()
+            && BackOffUtils.next(Sleeper.DEFAULT, backoff));
+
+        // Output any rows that failed along they way.
+        createRetryManagerResult
+            .getFailedRows()
+            .forEach(
+                tv -> o.get(failedRowsTag).outputWithTimestamp(tv.getValue(), 
tv.getTimestamp()));
+        
rowsSentToFailedRowsCollection.inc(createRetryManagerResult.getFailedRows().size());
+        BigQuerySinkMetrics.appendRowsRowStatusCounter(
+                BigQuerySinkMetrics.RowStatus.FAILED,
+                BigQuerySinkMetrics.PAYLOAD_TOO_LARGE,
+                shortTableId)
+            .inc(createRetryManagerResult.getFailedRows().size());
+
+        recordsAppended.inc(createRetryManagerResult.getRecordsAppended());
+        
createRetryManagerResult.getHistogramValues().forEach(appendSizeDistribution::update);
+
+        Instant now = Instant.now();
+
+        RetryManager<AppendRowsResponse, AppendRowsContext<DestinationT>> 
retryManager =
+            
Preconditions.checkStateNotNull(createRetryManagerResult.getRetryManager());
+        int numAppends = retryManager.getRemainingOperationCount();
+        Iterable<AppendRowsContext<DestinationT>> contexts = 
retryManager.getRemainingContexts();
+
+        if (numAppends > 0) {
+          initializeContexts.accept(contexts, false);
+          retryManager.run(true);
 
-      Consumer<AppendRowsContext<DestinationT>> onSuccess =
-          context -> {
-            AppendRowsResponse response = 
Preconditions.checkStateNotNull(context.getResult());
-            o.get(flushTag)
-                .output(
-                    KV.of(
-                        context.streamName,
-                        new Operation(
-                            context.offset + 
context.protoRows.getSerializedRowsCount() - 1,
-                            false)));
-            int flushedRows = context.protoRows.getSerializedRowsCount();
-            flushesScheduled.inc(flushedRows);
-            BigQuerySinkMetrics.reportSuccessfulRpcMetrics(
-                context, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, 
shortTableId);
-            BigQuerySinkMetrics.appendRowsRowStatusCounter(
-                    BigQuerySinkMetrics.RowStatus.SUCCESSFUL, 
BigQuerySinkMetrics.OK, shortTableId)
-                .inc(flushedRows);
-
-            if (successfulRowsTag != null) {
-              for (int i = 0; i < context.protoRows.getSerializedRowsCount(); 
++i) {
-                ByteString protoBytes = context.protoRows.getSerializedRows(i);
-                org.joda.time.Instant timestamp = context.timestamps.get(i);
-                o.get(successfulRowsTag)
-                    .outputWithTimestamp(
-                        appendClientInfo.get().toTableRow(protoBytes, 
successfulRowsPredicate),
-                        timestamp);
+          appendSplitDistribution.update(numAppends);
+          if (autoUpdateSchema) {
+            @Nullable
+            StreamAppendClient streamAppendClient = 
appendClientHolder.getStreamAppendClient();
+            TableSchema originalSchema = 
appendClientHolder.get().getTableSchema();
+
+            @Nullable
+            TableSchema updatedSchemaReturned =
+                (streamAppendClient != null) ? 
streamAppendClient.getUpdatedSchema() : null;
+            // Update the table schema and clear the append client.
+            if (updatedSchemaReturned != null) {
+              Optional<TableSchema> newSchema =
+                  TableSchemaUpdateUtils.getUpdatedSchema(originalSchema, 
updatedSchemaReturned);
+              if (newSchema.isPresent()) {
+                
APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey()));
+                LOG.debug(
+                    "Fetched updated schema for table {}:\n\t{}", tableId, 
updatedSchemaReturned);

Review Comment:
   I noticed we removed `APPEND_CLIENTS.put(...)` from here. Are we relying on 
the next process call to create the append client for `element.getKey()` ?



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


Reply via email to