Ver código fonte

HDFS-5356. MiniDFSCluster should close all open FileSystems when shutdown() (Contributed by Rakesh R)

(cherry picked from commit 018893e81ec1c43e6c79c77adec92c2edfb20cab)
Vinayakumar B 10 anos atrás
pai
commit
6ddb1bc857

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

@@ -14,6 +14,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    HDFS-5356. MiniDFSCluster should close all open FileSystems when shutdown()
+    (Rakesh R via vinayakumarb)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 28 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -61,6 +61,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -120,6 +121,7 @@ import org.apache.hadoop.util.ToolRunner;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 /**
  * This class creates a single-process DFS cluster for junit testing.
@@ -525,7 +527,8 @@ public class MiniDFSCluster {
   private boolean federation;
   private boolean checkExitOnShutdown = true;
   protected final int storagesPerDatanode;
-  
+  private Set<FileSystem> fileSystems = Sets.newHashSet();
+
   /**
    * A unique instance identifier for the cluster. This
    * is used to disambiguate HA filesystems in the case where
@@ -1709,6 +1712,13 @@ public class MiniDFSCluster {
    * Shutdown all the nodes in the cluster.
    */
   public void shutdown(boolean deleteDfsDir) {
+    shutdown(deleteDfsDir, true);
+  }
+
+  /**
+   * Shutdown all the nodes in the cluster.
+   */
+  public void shutdown(boolean deleteDfsDir, boolean closeFileSystem) {
     LOG.info("Shutting down the Mini HDFS Cluster");
     if (checkExitOnShutdown)  {
       if (ExitUtil.terminateCalled()) {
@@ -1718,6 +1728,16 @@ public class MiniDFSCluster {
         throw new AssertionError("Test resulted in an unexpected exit");
       }
     }
+    if (closeFileSystem) {
+      for (FileSystem fs : fileSystems) {
+        try {
+          fs.close();
+        } catch (IOException ioe) {
+          LOG.warn("Exception while closing file system", ioe);
+        }
+      }
+      fileSystems.clear();
+    }
     shutdownDataNodes();
     for (NameNodeInfo nnInfo : nameNodes) {
       if (nnInfo == null) continue;
@@ -2138,8 +2158,10 @@ public class MiniDFSCluster {
    * Get a client handle to the DFS cluster for the namenode at given index.
    */
   public DistributedFileSystem getFileSystem(int nnIndex) throws IOException {
-    return (DistributedFileSystem)FileSystem.get(getURI(nnIndex),
-        nameNodes[nnIndex].conf);
+    DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(
+        getURI(nnIndex), nameNodes[nnIndex].conf);
+    fileSystems.add(dfs);
+    return dfs;
   }
 
   /**
@@ -2147,7 +2169,9 @@ public class MiniDFSCluster {
    * This simulating different threads working on different FileSystem instances.
    */
   public FileSystem getNewFileSystemInstance(int nnIndex) throws IOException {
-    return FileSystem.newInstance(getURI(nnIndex), nameNodes[nnIndex].conf);
+    FileSystem dfs = FileSystem.newInstance(getURI(nnIndex), nameNodes[nnIndex].conf);
+    fileSystems.add(dfs);
+    return dfs;
   }
   
   /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -675,7 +675,7 @@ public class TestFileCreation {
 
       // restart cluster with the same namenode port as before.
       // This ensures that leases are persisted in fsimage.
-      cluster.shutdown();
+      cluster.shutdown(false, false);
       try {
         Thread.sleep(2*MAX_IDLE_TIME);
       } catch (InterruptedException e) {
@@ -687,7 +687,7 @@ public class TestFileCreation {
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
-      cluster.shutdown();
+      cluster.shutdown(false, false);
       try {
         Thread.sleep(5000);
       } catch (InterruptedException e) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -519,8 +519,8 @@ public class TestRenameWithSnapshots {
     File fsnAfter = new File(testDir, "dumptree_after");
     
     SnapshotTestHelper.dumpTree2File(fsdir, fsnBefore);
-    
-    cluster.shutdown();
+
+    cluster.shutdown(false, false);
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(REPL).build();
     cluster.waitActive();