Prechádzať zdrojové kódy

HDFS-3359. DFSClient.close should close cached sockets. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1333625 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 rokov pred
rodič
commit
9c5ec67761

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

@@ -471,6 +471,8 @@ Release 2.0.0 - UNRELEASED
     HDFS-3332. NullPointerException in DN when directoryscanner is trying to 
     report bad blocks. (Amith D K via umamahesh)
 
+    HDFS-3359. DFSClient.close should close cached sockets. (todd)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -563,6 +563,7 @@ public class DFSClient implements java.io.Closeable {
   void abort() {
     clientRunning = false;
     closeAllFilesBeingWritten(true);
+    socketCache.clear();
     closeConnectionToNamenode();
   }
 
@@ -600,6 +601,7 @@ public class DFSClient implements java.io.Closeable {
   public synchronized void close() throws IOException {
     if(clientRunning) {
       closeAllFilesBeingWritten(false);
+      socketCache.clear();
       clientRunning = false;
       leaserenewer.closeClient(this);
       // close connections to the namenode

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -85,6 +85,7 @@ public class TestDistributedFileSystem {
   /**
    * Tests DFSClient.close throws no ConcurrentModificationException if 
    * multiple files are open.
+   * Also tests that any cached sockets are closed. (HDFS-3359)
    */
   @Test
   public void testDFSClose() throws Exception {
@@ -94,11 +95,23 @@ public class TestDistributedFileSystem {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       FileSystem fileSys = cluster.getFileSystem();
       
-      // create two files
+      // create two files, leaving them open
       fileSys.create(new Path("/test/dfsclose/file-0"));
       fileSys.create(new Path("/test/dfsclose/file-1"));
+      
+      // create another file, close it, and read it, so
+      // the client gets a socket in its SocketCache
+      Path p = new Path("/non-empty-file");
+      DFSTestUtil.createFile(fileSys, p, 1L, (short)1, 0L);
+      DFSTestUtil.readFile(fileSys, p);
+      
+      DFSClient client = ((DistributedFileSystem)fileSys).dfs;
+      SocketCache cache = client.socketCache;
+      assertEquals(1, cache.size());
 
       fileSys.close();
+      
+      assertEquals(0, cache.size());
     } finally {
       if (cluster != null) {cluster.shutdown();}
     }