gemini-code-assist[bot] commented on code in PR #38264:
URL: https://github.com/apache/beam/pull/38264#discussion_r3149558552


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientCache.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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 get(KeyT key, Callable<AppendClientInfo> loader) 
throws Exception {
+    return appendCache.get(key, wrapWithPin(loader));
+  }

Review Comment:
   ![high](https://www.gstatic.com/codereviewagent/high-priority.svg)
   
   This method should be `synchronized` to maintain the atomicity protocol 
described in the class Javadoc. Without synchronization, a concurrent access 
via this method could trigger cache maintenance and evict an entry that another 
thread is currently pinning in `getAndPin`.
   
   ```suggestion
     public synchronized AppendClientInfo get(KeyT key, 
Callable<AppendClientInfo> loader)
         throws Exception {
       return appendCache.get(key, wrapWithPin(loader));
     }
   ```



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientCache.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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 get(KeyT key, Callable<AppendClientInfo> loader) 
throws Exception {
+    return appendCache.get(key, wrapWithPin(loader));
+  }
+
+  public AppendClientInfo putAndPin(KeyT key, Callable<AppendClientInfo> 
loader) throws Exception {
+    synchronized (this) {
+      AppendClientInfo info = wrapWithPin(loader).call();
+      appendCache.put(key, info);
+      info.pinAppendClient();
+      return info;
+    }
+  }
+
+  public AppendClientInfo put(KeyT key, Callable<AppendClientInfo> loader) 
throws Exception {
+    AppendClientInfo info = wrapWithPin(loader).call();
+    appendCache.put(key, info);
+    return info;
+  }

Review Comment:
   ![high](https://www.gstatic.com/codereviewagent/high-priority.svg)
   
   This method should be `synchronized` to ensure that the cache access is 
atomic relative to other operations like `getAndPin` and `invalidate`. This 
prevents races where an entry might be evicted or invalidated while being 
updated.
   
   ```suggestion
     public synchronized AppendClientInfo put(KeyT key, 
Callable<AppendClientInfo> loader)
         throws Exception {
       AppendClientInfo info = wrapWithPin(loader).call();
       appendCache.put(key, info);
       return info;
     }
   ```



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientCache.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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 get(KeyT key, Callable<AppendClientInfo> loader) 
throws Exception {
+    return appendCache.get(key, wrapWithPin(loader));
+  }
+
+  public AppendClientInfo putAndPin(KeyT key, Callable<AppendClientInfo> 
loader) throws Exception {
+    synchronized (this) {
+      AppendClientInfo info = wrapWithPin(loader).call();
+      appendCache.put(key, info);
+      info.pinAppendClient();
+      return info;
+    }
+  }
+
+  public AppendClientInfo put(KeyT key, Callable<AppendClientInfo> loader) 
throws Exception {
+    AppendClientInfo info = wrapWithPin(loader).call();
+    appendCache.put(key, info);
+    return info;
+  }
+
+  public void invalidate(KeyT key, AppendClientInfo expectedClient) {
+    // The default stream is cached across multiple different DoFns. If they 
all try
+    // and
+    // invalidate, then we can get races between threads invalidating and 
recreating
+    // streams. For this reason,
+    // we check to see that the cache still contains the object we created 
before
+    // invalidating (in case another
+    // thread has already invalidated and recreated the stream).
+    synchronized (this) {
+      AppendClientInfo cachedAppendClient = appendCache.getIfPresent(key);
+      if (cachedAppendClient != null
+          && System.identityHashCode(cachedAppendClient)
+              == System.identityHashCode(expectedClient)) {
+        appendCache.invalidate(key);
+      }
+    }
+  }
+
+  public void invalidate(KeyT key) {
+    synchronized (this) {
+      appendCache.invalidate(key);
+    }
+  }
+
+  public void tickle(KeyT key) {
+    appendCache.getIfPresent(key);
+  }

Review Comment:
   ![high](https://www.gstatic.com/codereviewagent/high-priority.svg)
   
   The `tickle` method should be `synchronized`. Since it performs a cache 
access (`getIfPresent`), it can trigger maintenance and eviction. To protect 
the atomicity of the pinning protocol used in `getAndPin`, all cache access 
methods must be synchronized on the same monitor.
   
   ```suggestion
     public synchronized void tickle(KeyT key) {
       appendCache.getIfPresent(key);
     }
   ```



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java:
##########
@@ -199,4 +219,38 @@ public TableRow toTableRow(ByteString protoBytes, 
Predicate<String> includeField
       throw new RuntimeException(e);
     }
   }
+
+  public void pinAppendClient() {
+    BigQueryServices.StreamAppendClient client =
+        Preconditions.checkStateNotNull(getStreamAppendClient());
+    client.pin();
+  }
+
+  public void unpinAppendClient(@Nullable ExecutorService executor) {
+    BigQueryServices.StreamAppendClient client =
+        Preconditions.checkStateNotNull(getStreamAppendClient());
+    if (executor != null) {
+      runAsyncIgnoreFailure(executor, client::unpin);
+    } else {
+      client.unpin();
+    }
+  }
+
+  @SuppressWarnings({"FutureReturnValueIgnored"})
+  private static void runAsyncIgnoreFailure(ExecutorService executor, 
ThrowingRunnable task) {
+    executor.submit(
+        () -> {
+          try {
+            task.run();
+          } catch (Throwable e) {
+            String msg =
+                e.toString()
+                    + "\n"
+                    + Arrays.stream(e.getStackTrace())
+                        .map(StackTraceElement::toString)
+                        .collect(Collectors.joining("\n"));
+            System.err.println("Exception happened while executing async task. 
Ignoring: " + msg);

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   Avoid using `System.err.println` for error reporting. Use the class logger 
instead. Additionally, let the logger handle the exception object directly 
rather than manually formatting the stack trace.
   
   ```java
               LOG.error("Exception happened while executing async task. 
Ignoring", e);
   ```



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java:
##########
@@ -554,6 +553,214 @@ private CreateRetryManagerResult<DestinationT> 
createRetryManager(
           retryManager, failedRows, recordsAppended, histogramUpdates);
     }
 
+    private void handleAppendFailure(
+        Iterable<AppendRowsContext<DestinationT>> failedContexts,
+        TableReference tableReference,
+        String shortTableId,
+        AppendClientInfo appendClientInfo,
+        Callable<Boolean> tryCreateTable,
+        BiConsumer<Iterable<AppendRowsContext<DestinationT>>, Boolean> 
initializeContexts,
+        Consumer<Iterable<AppendRowsContext<DestinationT>>> clearClients,
+        ValueState<Long> streamOffset,
+        MultiOutputReceiver o) {
+      // 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) {
+            ByteString protoBytes = 
failedContext.protoRows.getSerializedRows(failedIndex);
+            failedRow = appendClientInfo.toTableRow(protoBytes, 
Predicates.alwaysTrue());
+          }
+          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));
+          }
+        }
+        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();
+        }
+        streamOffset.write(offset);
+        return;
+      }
+
+      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");
+        }
+      }
+      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 {}",

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   Typo: 'BigTable' should be 'BigQuery' in this context, as this class handles 
BigQuery Storage API writes.
   
   ```java
               "Vortex failed stream open due to incompatible fields. This is 
likely because the BigQuery "
   ```



##########
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:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   The manual formatting of the stack trace is unnecessary and inefficient. 
SLF4J loggers can handle exceptions directly, which includes printing the stack 
trace. This also fixes a typo ('whilw' -> 'while').
   
   ```java
           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/StorageApiWritesShardedRecords.java:
##########
@@ -149,44 +141,12 @@ public class StorageApiWritesShardedRecords<DestinationT 
extends @NonNull Object
   private final Coder<TableRow> succussfulRowsCoder;
 
   private final TupleTag<KV<String, Operation>> flushTag = new 
TupleTag<>("flushTag");
-  private static final ExecutorService closeWriterExecutor = 
Executors.newCachedThreadPool();
-
-  private static final Cache<KV<String, ShardedKey<?>>, AppendClientInfo> 
APPEND_CLIENTS =
-      CacheBuilder.newBuilder()
-          .expireAfterAccess(5, TimeUnit.MINUTES)
-          .removalListener(
-              (RemovalNotification<KV<String, ShardedKey<?>>, 
AppendClientInfo> removal) -> {
-                final @Nullable AppendClientInfo appendClientInfo = 
removal.getValue();
-                if (appendClientInfo != null) {
-                  appendClientInfo.close();
-                }
-              })
-          .build();
 
-  static void clearCache() {
-    APPEND_CLIENTS.invalidateAll();
-  }
+  private static AppendClientCache<KV<String, ShardedKey<?>>> APPEND_CLIENTS =
+      new AppendClientCache<>(Duration.standardMinutes(5));

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   The `APPEND_CLIENTS` cache should be declared as `final` to ensure it is not 
reassigned and to follow best practices for static constants.
   
   ```suggestion
     private static final AppendClientCache<KV<String, ShardedKey<?>>> 
APPEND_CLIENTS =
         new AppendClientCache<>(Duration.standardMinutes(5));
   ```



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