Explorar el Código

HDFS-3365. Enable users to disable socket caching in DFS client configuration. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1335221 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon hace 13 años
padre
commit
4fa0dd2278

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -283,6 +283,9 @@ Release 2.0.0 - UNRELEASED
     HDFS-2617. Replaced Kerberized SSL for image transfer and fsck
     with SPNEGO-based solution. (jghoman, tucu, and atm via eli)
 
+    HDFS-3365. Enable users to disable socket caching in DFS client
+    configuration (todd)
+
   OPTIMIZATIONS
 
     HDFS-2477. Optimize computing the diff between a block report and the

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/SocketCache.java

@@ -47,6 +47,9 @@ class SocketCache {
   public SocketCache(int capacity) {
     multimap = LinkedListMultimap.create();
     this.capacity = capacity;
+    if (capacity <= 0) {
+      LOG.debug("SocketCache disabled in configuration.");
+    }
   }
 
   /**
@@ -55,6 +58,10 @@ class SocketCache {
    * @return  A socket with unknown state, possibly closed underneath. Or null.
    */
   public synchronized Socket get(SocketAddress remote) {
+    if (capacity <= 0) { // disabled
+      return null;
+    }
+    
     List<Socket> socklist = multimap.get(remote);
     if (socklist == null) {
       return null;
@@ -76,6 +83,12 @@ class SocketCache {
    * @param sock socket not used by anyone.
    */
   public synchronized void put(Socket sock) {
+    if (capacity <= 0) {
+      // Cache disabled.
+      IOUtils.closeSocket(sock);
+      return;
+    }
+    
     Preconditions.checkNotNull(sock);
 
     SocketAddress remoteAddr = sock.getRemoteSocketAddress();

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java

@@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.io.IOUtils;
 
 import org.apache.hadoop.security.token.Token;
 import org.junit.Test;
@@ -230,6 +232,33 @@ public class TestConnCache {
 
     in.close();
   }
+  
+  /**
+   * Test that the socket cache can be disabled by setting the capacity to
+   * 0. Regression test for HDFS-3365.
+   */
+  @Test
+  public void testDisableCache() throws IOException {
+    LOG.info("Starting testDisableCache()");
+
+    // Reading with the normally configured filesystem should
+    // cache a socket.
+    DFSTestUtil.readFile(fs, testFile);
+    assertEquals(1, ((DistributedFileSystem)fs).dfs.socketCache.size());
+    
+    // Configure a new instance with no caching, ensure that it doesn't
+    // cache anything
+    Configuration confWithoutCache = new Configuration(fs.getConf());
+    confWithoutCache.setInt(
+        DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY, 0);
+    FileSystem fsWithoutCache = FileSystem.newInstance(confWithoutCache);
+    try {
+      DFSTestUtil.readFile(fsWithoutCache, testFile);
+      assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.socketCache.size());
+    } finally {
+      fsWithoutCache.close();
+    }
+  }
 
   @AfterClass
   public static void teardownCluster() throws Exception {