Skip to content

Properly close Storage API batch connections #31710

New issue

Have a question about this project? # for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “#”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? # to your account

Merged
merged 3 commits into from
Jun 28, 2024
Merged
Show file tree
Hide file tree
Changes from all 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
Expand Up @@ -30,6 +30,8 @@
import java.util.function.Consumer;
import java.util.function.Supplier;
import javax.annotation.Nullable;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Metrics;

/**
* Container class used by {@link StorageApiWritesShardedRecords} and {@link
Expand All @@ -38,6 +40,9 @@
*/
@AutoValue
abstract class AppendClientInfo {
private final Counter activeConnections =
Metrics.counter(AppendClientInfo.class, "activeConnections");

abstract @Nullable BigQueryServices.StreamAppendClient getStreamAppendClient();

abstract TableSchema getTableSchema();
Expand Down Expand Up @@ -114,19 +119,21 @@ public AppendClientInfo withAppendClient(
return this;
} else {
String streamName = getStreamName.get();
return toBuilder()
.setStreamName(streamName)
.setStreamAppendClient(
writeStreamService.getStreamAppendClient(
streamName, getDescriptor(), useConnectionPool, missingValueInterpretation))
.build();
BigQueryServices.StreamAppendClient client =
writeStreamService.getStreamAppendClient(
streamName, getDescriptor(), useConnectionPool, missingValueInterpretation);

activeConnections.inc();

return toBuilder().setStreamName(streamName).setStreamAppendClient(client).build();
}
}

public void close() {
BigQueryServices.StreamAppendClient client = getStreamAppendClient();
if (client != null) {
getCloseAppendClient().accept(client);
activeConnections.dec();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,11 @@ void teardown() {
if (client != null) {
runAsyncIgnoreFailure(closeWriterExecutor, client::unpin);
}
// if this is a PENDING stream, we won't be using it again after cleaning up this
// destination state, so clear it from the cache
if (!useDefaultStream) {
APPEND_CLIENTS.invalidate(streamName);
}
appendClientInfo = null;
}
}
Expand Down
Loading