mxm commented on a change in pull request #11108: [BEAM-9490] Guard referencing
for environment expiration via a lock
URL: https://github.com/apache/beam/pull/11108#discussion_r391700263
##########
File path:
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
##########
@@ -161,29 +167,33 @@ public static DefaultJobBundleFactory create(
this.stageIdGenerator = stageIdGenerator;
this.environmentExpirationMillis = getEnvironmentExpirationMillis(jobInfo);
this.loadBalanceBundles = shouldLoadBalanceBundles(jobInfo);
+ this.environmentCacheLocks =
createEnvironmentCacheLocks(getMaxEnvironmentClients(jobInfo));
this.environmentCaches =
createEnvironmentCaches(serverFactory -> serverInfo,
getMaxEnvironmentClients(jobInfo));
this.availableCachesSemaphore = new Semaphore(environmentCaches.size(),
true);
this.availableCaches = new LinkedBlockingDeque<>(environmentCaches);
- this.evictedActiveClients = Sets.newIdentityHashSet();
+ this.evictedActiveClients = Sets.newConcurrentHashSet();
+ }
+
+ private ImmutableList<Lock> createEnvironmentCacheLocks(int count) {
+ ImmutableList.Builder<Lock> locksForCaches = ImmutableList.builder();
+ for (int i = 0; i < count; i++) {
+ final Lock refLock;
+ if (environmentExpirationMillis > 0) {
+ // The lock ensures there is no race condition between expiring an
environment and a client
+ // still attempting to use it, hence referencing it.
+ refLock = new ReentrantLock(true);
+ } else {
+ refLock = NoopLock.get();
+ }
+ locksForCaches.add(refLock);
+ }
+ return locksForCaches.build();
}
private ImmutableList<LoadingCache<Environment, WrappedSdkHarnessClient>>
createEnvironmentCaches(
ThrowingFunction<ServerFactory, ServerInfo> serverInfoCreator, int
count) {
- CacheBuilder<Environment, WrappedSdkHarnessClient> builder =
- CacheBuilder.newBuilder()
- .removalListener(
- (RemovalNotification<Environment, WrappedSdkHarnessClient>
notification) -> {
- WrappedSdkHarnessClient client = notification.getValue();
- int refCount = client.unref();
- if (refCount > 0) {
- LOG.warn(
- "Expiring environment {} with {} remaining bundle
references. Taking note to clean it up during shutdown if the references are
not removed by then.",
- notification.getKey(),
- refCount);
- evictedActiveClients.add(client);
- }
- });
+ CacheBuilder builder = CacheBuilder.newBuilder();
Review comment:
Unfortunately, we lose the type infos here. There is no way without calling
a builder method to get the correct type.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services