Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -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();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

It is recommended to use a named ThreadFactory with daemon threads for the ExecutorService. This improves observability in thread dumps and ensures that the threads do not prevent the JVM from exiting if the process is shutting down.

  private final ExecutorService closeWriterExecutor =
      Executors.newCachedThreadPool(
          new org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder()
              .setDaemon(true)
              .setNameFormat("AppendClientCache-closer-%d")
              .build());


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;
}
Comment on lines +85 to +89
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

Using synchronized(this) on the entire getAndPin method creates a global bottleneck for all cache accesses. Since the loader (which creates the StreamAppendClient) can be slow due to network calls, this will block all other threads trying to access the cache for different keys. Consider using a more granular locking mechanism, such as Guava's Striped locks, to allow concurrent access for different keys.

}

public AppendClientInfo putAndPin(KeyT key, Callable<AppendClientInfo> loader) throws Exception {
synchronized (this) {
AppendClientInfo info = wrapWithPin(loader).call();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to unpin any existing AppendClients?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure I understand - do you mean unpin other clients when inserting this one?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, unpinning a client if it already exists for this key

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, that's actually a good point. The only place we call it is immediately after a call to invalidate, but there's a race condition there where somebody else could sneak in.

I don't think this could actually cause problems, since the pin owned by the DoFn is always released based on the instance variable (this.appendClientInfo), so even if we "lose" the cache entry we won't lose the pin. However to make things a bit clearer, I got rid of this method.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry I think I just got mixed up on the unpinning logic.
I thought we were leaking resources by not unpinning the consuming processElements, but I just did another pass and it looks fine to me.

appendCache.put(key, info);
info.pinAppendClient();
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) {
synchronized (this) {
appendCache.getIfPresent(key);
}
}
Comment on lines +125 to +129
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

high

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.

Suggested change
public void tickle(KeyT key) {
appendCache.getIfPresent(key);
}
public synchronized void tickle(KeyT key) {
appendCache.getIfPresent(key);
}


public void clear() {
synchronized (this) {
appendCache.invalidateAll();
}
}

public void unpinAsync(AppendClientInfo appendClientInfo) {
appendClientInfo.unpinAppendClient(closeWriterExecutor);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,13 +27,19 @@
import com.google.protobuf.Descriptors;
import com.google.protobuf.DynamicMessage;
import com.google.protobuf.Message;
import java.util.function.Consumer;
import java.util.Arrays;
import java.util.concurrent.ExecutorService;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import org.apache.beam.sdk.function.ThrowingConsumer;
import org.apache.beam.sdk.function.ThrowingRunnable;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Container class used by {@link StorageApiWritesShardedRecords} and {@link
Expand All @@ -42,14 +48,16 @@
*/
@AutoValue
abstract class AppendClientInfo {
private static final Logger LOG = LoggerFactory.getLogger(AppendClientInfo.class);

private final Counter activeStreamAppendClients =
Metrics.counter(AppendClientInfo.class, "activeStreamAppendClients");

abstract @Nullable BigQueryServices.StreamAppendClient getStreamAppendClient();

abstract TableSchema getTableSchema();

abstract Consumer<BigQueryServices.StreamAppendClient> getCloseAppendClient();
abstract ThrowingConsumer<Exception, BigQueryServices.StreamAppendClient> getCloseAppendClient();

abstract com.google.api.services.bigquery.model.TableSchema getJsonTableSchema();

Expand All @@ -65,7 +73,8 @@ abstract static class Builder {

abstract Builder setTableSchema(TableSchema value);

abstract Builder setCloseAppendClient(Consumer<BigQueryServices.StreamAppendClient> value);
abstract Builder setCloseAppendClient(
ThrowingConsumer<Exception, BigQueryServices.StreamAppendClient> value);

abstract Builder setJsonTableSchema(com.google.api.services.bigquery.model.TableSchema value);

Expand All @@ -83,7 +92,7 @@ abstract static class Builder {
static AppendClientInfo of(
TableSchema tableSchema,
DescriptorProtos.DescriptorProto descriptor,
Consumer<BigQueryServices.StreamAppendClient> closeAppendClient)
ThrowingConsumer<Exception, BigQueryServices.StreamAppendClient> closeAppendClient)
throws Exception {
return new AutoValue_AppendClientInfo.Builder()
.setTableSchema(tableSchema)
Expand All @@ -97,7 +106,7 @@ static AppendClientInfo of(

static AppendClientInfo of(
TableSchema tableSchema,
Consumer<BigQueryServices.StreamAppendClient> closeAppendClient,
ThrowingConsumer<Exception, BigQueryServices.StreamAppendClient> closeAppendClient,
boolean includeCdcColumns)
throws Exception {
return of(
Expand Down Expand Up @@ -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);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

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').

        LOG.warn("Caught exception while trying to close append client. Ignoring", e);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

The manual formatting of the stack trace is redundant and inefficient. SLF4J's logger can handle exceptions directly, which includes printing the stack trace. Additionally, there is a typo in the log message ('whilw' instead of 'while').

        LOG.warn("Caught exception while trying to close append client. Ignoring", e);

Comment thread
ahmedabu98 marked this conversation as resolved.
Outdated
}
activeStreamAppendClients.dec();
}
}
Expand Down Expand Up @@ -199,4 +219,32 @@ 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) {
LOG.info("Exception happened while executing async task. Ignoring: ", e);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

Logging an exception at INFO level is unusual. Even if the failure is ignored, it is generally better to log it at WARN or ERROR level to make it more visible in logs when troubleshooting background tasks.

Suggested change
LOG.info("Exception happened while executing async task. Ignoring: ", e);
LOG.warn("Exception happened while executing async task. Ignoring: ", e);

}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ default long getInflightWaitSeconds() {
/**
* Unpin this object. If the object has been closed, this will release any underlying resources.
*/
void unpin() throws Exception;
void unpin();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1617,7 +1617,7 @@ public void pin() {
}

@Override
public void unpin() throws Exception {
public void unpin() {
boolean closeWriter;
synchronized (this) {
Preconditions.checkState(pins > 0, "Tried to unpin when pins==0");
Expand Down
Loading
Loading