Skip to content

Commit

Permalink
[apache#772][0.7] fix(kerberos): cache proxy user ugi to avoid memory…
Browse files Browse the repository at this point in the history
… leak (apache#773) (apache#824)

1. To avoid memory leak by caching of proxy user UGI.

Fix: apache#772

The Hadoop filesystem instance will be created too many time in cache, which will cause the shuffle server memory leak.

As we know, the filesystem cache's key is built by the scheme、authority and UGI. The scheme and authority are not changed every time. But for UGI, if we invoke the createProxyUser, it will always create a new one, that means the every invoking `Filesystem.get()`, it will be cached due to different key.

No.

1. Existing UTs
2. Added tests
  • Loading branch information
zuston authored and xianjingfeng committed Jun 20, 2023
1 parent 4cdfbec commit 926c314
Showing 1 changed file with 23 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,13 @@

import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Maps;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
Expand All @@ -38,6 +41,12 @@ public class HadoopSecurityContext implements SecurityContext {
private UserGroupInformation loginUgi;
private ScheduledExecutorService refreshScheduledExecutor;

// The purpose of the proxy user ugi cache is to prevent the creation of
// multiple cache keys for the same user, scheme, and authority in the Hadoop filesystem.
// Without this cache, large amounts of unnecessary filesystem instances could be stored in memory,
// leading to potential memory leaks. For more information on this issue, refer to #706.
private Map<String, UserGroupInformation> proxyUserUgiPool;

public HadoopSecurityContext(
String krb5ConfPath,
String keytabFile,
Expand Down Expand Up @@ -72,6 +81,7 @@ public HadoopSecurityContext(
refreshIntervalSec,
refreshIntervalSec,
TimeUnit.SECONDS);
proxyUserUgiPool = Maps.newConcurrentMap();
}

private void authRefresh() {
Expand All @@ -91,8 +101,10 @@ public <T> T runSecured(String user, Callable<T> securedCallable) throws Excepti

// Run with the proxy user.
if (!user.equals(loginUgi.getShortUserName())) {
UserGroupInformation proxyUserUgi =
proxyUserUgiPool.computeIfAbsent(user, x -> UserGroupInformation.createProxyUser(x, loginUgi));
return executeWithUgiWrapper(
UserGroupInformation.createProxyUser(user, loginUgi),
proxyUserUgi,
securedCallable
);
}
Expand All @@ -110,10 +122,20 @@ private <T> T executeWithUgiWrapper(UserGroupInformation ugi, Callable<T> callab
return ugi.doAs((PrivilegedExceptionAction<T>) callable::call);
}

// Only for tests
@VisibleForTesting
Map<String, UserGroupInformation> getProxyUserUgiPool() {
return proxyUserUgiPool;
}

@Override
public void close() throws IOException {
if (refreshScheduledExecutor != null) {
refreshScheduledExecutor.shutdown();
}
if (proxyUserUgiPool != null) {
proxyUserUgiPool.clear();
proxyUserUgiPool = null;
}
}
}

0 comments on commit 926c314

Please sign in to comment.