|
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.nfs.nfs3;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -27,59 +28,81 @@ import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+import com.google.common.cache.CacheBuilder;
|
|
|
+import com.google.common.cache.CacheLoader;
|
|
|
+import com.google.common.cache.LoadingCache;
|
|
|
+import com.google.common.cache.RemovalListener;
|
|
|
+import com.google.common.cache.RemovalNotification;
|
|
|
+
|
|
|
/**
|
|
|
* A cache saves DFSClient objects for different users
|
|
|
*/
|
|
|
-public class DFSClientCache {
|
|
|
- static final Log LOG = LogFactory.getLog(DFSClientCache.class);
|
|
|
- private final LruCache<String, DFSClient> lruTable;
|
|
|
+class DFSClientCache {
|
|
|
+ private static final Log LOG = LogFactory.getLog(DFSClientCache.class);
|
|
|
+ /**
|
|
|
+ * Cache that maps User id to corresponding DFSClient.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ final LoadingCache<String, DFSClient> clientCache;
|
|
|
+
|
|
|
+ final static int DEFAULT_DFS_CLIENT_CACHE_SIZE = 256;
|
|
|
+
|
|
|
private final Configuration config;
|
|
|
|
|
|
- public DFSClientCache(Configuration config) {
|
|
|
- // By default, keep 256 DFSClient instance for 256 active users
|
|
|
- this(config, 256);
|
|
|
+ DFSClientCache(Configuration config) {
|
|
|
+ this(config, DEFAULT_DFS_CLIENT_CACHE_SIZE);
|
|
|
}
|
|
|
|
|
|
- public DFSClientCache(Configuration config, int size) {
|
|
|
- lruTable = new LruCache<String, DFSClient>(size);
|
|
|
+ DFSClientCache(Configuration config, int clientCache) {
|
|
|
this.config = config;
|
|
|
+ this.clientCache = CacheBuilder.newBuilder()
|
|
|
+ .maximumSize(clientCache)
|
|
|
+ .removalListener(clientRemovealListener())
|
|
|
+ .build(clientLoader());
|
|
|
}
|
|
|
|
|
|
- public void put(String uname, DFSClient client) {
|
|
|
- lruTable.put(uname, client);
|
|
|
- }
|
|
|
+ private CacheLoader<String, DFSClient> clientLoader() {
|
|
|
+ return new CacheLoader<String, DFSClient>() {
|
|
|
+ @Override
|
|
|
+ public DFSClient load(String userName) throws Exception {
|
|
|
+ UserGroupInformation ugi = UserGroupInformation
|
|
|
+ .createRemoteUser(userName);
|
|
|
|
|
|
- synchronized public DFSClient get(String uname) {
|
|
|
- DFSClient client = lruTable.get(uname);
|
|
|
- if (client != null) {
|
|
|
- return client;
|
|
|
- }
|
|
|
+ // Guava requires CacheLoader never returns null.
|
|
|
+ return ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
|
|
|
+ public DFSClient run() throws IOException {
|
|
|
+ return new DFSClient(NameNode.getAddress(config), config);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+ };
|
|
|
+ }
|
|
|
|
|
|
- // Not in table, create one.
|
|
|
- try {
|
|
|
- UserGroupInformation ugi = UserGroupInformation.createRemoteUser(uname);
|
|
|
- client = ugi.doAs(new PrivilegedExceptionAction<DFSClient>() {
|
|
|
- public DFSClient run() throws IOException {
|
|
|
- return new DFSClient(NameNode.getAddress(config), config);
|
|
|
+ private RemovalListener<String, DFSClient> clientRemovealListener() {
|
|
|
+ return new RemovalListener<String, DFSClient>() {
|
|
|
+ @Override
|
|
|
+ public void onRemoval(RemovalNotification<String, DFSClient> notification) {
|
|
|
+ DFSClient client = notification.getValue();
|
|
|
+ try {
|
|
|
+ client.close();
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn(String.format(
|
|
|
+ "IOException when closing the DFSClient(%s), cause: %s", client,
|
|
|
+ e));
|
|
|
}
|
|
|
- });
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.error("Create DFSClient failed for user:" + uname);
|
|
|
- e.printStackTrace();
|
|
|
+ }
|
|
|
+ };
|
|
|
+ }
|
|
|
|
|
|
- } catch (InterruptedException e) {
|
|
|
- e.printStackTrace();
|
|
|
+ DFSClient get(String userName) {
|
|
|
+ DFSClient client = null;
|
|
|
+ try {
|
|
|
+ client = clientCache.get(userName);
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ LOG.error("Failed to create DFSClient for user:" + userName + " Cause:"
|
|
|
+ + e);
|
|
|
}
|
|
|
- // Add new entry
|
|
|
- lruTable.put(uname, client);
|
|
|
return client;
|
|
|
}
|
|
|
-
|
|
|
- public int usedSize() {
|
|
|
- return lruTable.usedSize();
|
|
|
- }
|
|
|
-
|
|
|
- public boolean containsKey(String key) {
|
|
|
- return lruTable.containsKey(key);
|
|
|
- }
|
|
|
}
|