Selaa lähdekoodia

HADOOP-8562. Merge changes r1454616 for HDFS-4573, r1454889 for HADOOP-8973, r1455360 for HDFS-4572, r1455605 for HDFS-4287 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1485892 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 vuotta sitten
vanhempi
commit
830909a910
14 muutettua tiedostoa jossa 244 lisäystä ja 130 poistoa
  1. 99 30
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java
  2. 20 23
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/Server.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  5. 30 15
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java
  6. 24 10
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java
  7. 4 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestHostnameFilter.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java
  9. 2 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java
  10. 7 2
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  12. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  13. 15 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
  14. 35 28
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

+ 99 - 30
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java

@@ -23,11 +23,10 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
 
 /**
  * Class that provides utility functions for checking disk problem
@@ -36,10 +35,16 @@ import org.apache.hadoop.fs.permission.FsPermission;
 @InterfaceStability.Unstable
 public class DiskChecker {
 
+  private static final long SHELL_TIMEOUT = 10 * 1000;
+
   public static class DiskErrorException extends IOException {
     public DiskErrorException(String msg) {
       super(msg);
     }
+
+    public DiskErrorException(String msg, Throwable cause) {
+      super(msg, cause);
+    }
   }
     
   public static class DiskOutOfSpaceException extends IOException {
@@ -85,25 +90,11 @@ public class DiskChecker {
    * @throws DiskErrorException
    */
   public static void checkDir(File dir) throws DiskErrorException {
-    if (!mkdirsWithExistsCheck(dir))
+    if (!mkdirsWithExistsCheck(dir)) {
       throw new DiskErrorException("Can not create directory: "
                                    + dir.toString());
-
-    if (!dir.isDirectory())
-      throw new DiskErrorException("Not a directory: "
-                                   + dir.toString());
-
-    if (!dir.canRead())
-      throw new DiskErrorException("Directory is not readable: "
-                                   + dir.toString());
-
-    if (!dir.canWrite())
-      throw new DiskErrorException("Directory is not writable: "
-                                   + dir.toString());
-
-    if (!dir.canExecute())
-      throw new DiskErrorException("Directory is not executable: "
-	  + dir.toString());
+    }
+    checkDirAccess(dir);
   }
 
   /**
@@ -152,24 +143,102 @@ public class DiskChecker {
                               FsPermission expected)
   throws DiskErrorException, IOException {
     mkdirsWithExistsAndPermissionCheck(localFS, dir, expected);
+    checkDirAccess(localFS.pathToFile(dir));
+  }
 
-    FileStatus stat = localFS.getFileStatus(dir);
-    FsPermission actual = stat.getPermission();
+  /**
+   * Checks that the given file is a directory and that the current running
+   * process can read, write, and execute it.
+   * 
+   * @param dir File to check
+   * @throws DiskErrorException if dir is not a directory, not readable, not
+   *   writable, or not executable
+   */
+  private static void checkDirAccess(File dir) throws DiskErrorException {
+    if (!dir.isDirectory()) {
+      throw new DiskErrorException("Not a directory: "
+                                   + dir.toString());
+    }
 
-    if (!stat.isDirectory())
-      throw new DiskErrorException("not a directory: "+ dir.toString());
+    if (Shell.WINDOWS) {
+      checkAccessByFileSystemInteraction(dir);
+    } else {
+      checkAccessByFileMethods(dir);
+    }
+  }
 
-    FsAction user = actual.getUserAction();
-    if (!user.implies(FsAction.READ))
-      throw new DiskErrorException("directory is not readable: "
+  /**
+   * Checks that the current running process can read, write, and execute the
+   * given directory by using methods of the File object.
+   * 
+   * @param dir File to check
+   * @throws DiskErrorException if dir is not readable, not writable, or not
+   *   executable
+   */
+  private static void checkAccessByFileMethods(File dir)
+      throws DiskErrorException {
+    if (!dir.canRead()) {
+      throw new DiskErrorException("Directory is not readable: "
+                                   + dir.toString());
+    }
+
+    if (!dir.canWrite()) {
+      throw new DiskErrorException("Directory is not writable: "
                                    + dir.toString());
+    }
 
-    if (!user.implies(FsAction.WRITE))
-      throw new DiskErrorException("directory is not writable: "
+    if (!dir.canExecute()) {
+      throw new DiskErrorException("Directory is not executable: "
                                    + dir.toString());
+    }
+  }
 
-    if (!user.implies(FsAction.EXECUTE))
-      throw new DiskErrorException("directory is not listable: "
+  /**
+   * Checks that the current running process can read, write, and execute the
+   * given directory by attempting each of those operations on the file system.
+   * This method contains several workarounds to known JVM bugs that cause
+   * File.canRead, File.canWrite, and File.canExecute to return incorrect results
+   * on Windows with NTFS ACLs.  See:
+   * http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6203387
+   * These bugs are supposed to be fixed in JDK7.
+   * 
+   * @param dir File to check
+   * @throws DiskErrorException if dir is not readable, not writable, or not
+   *   executable
+   */
+  private static void checkAccessByFileSystemInteraction(File dir)
+      throws DiskErrorException {
+    // Make sure we can read the directory by listing it.
+    if (dir.list() == null) {
+      throw new DiskErrorException("Directory is not readable: "
                                    + dir.toString());
+    }
+
+    // Make sure we can write to the directory by creating a temp file in it.
+    try {
+      File tempFile = File.createTempFile("checkDirAccess", null, dir);
+      if (!tempFile.delete()) {
+        throw new DiskErrorException("Directory is not writable: "
+                                     + dir.toString());
+      }
+    } catch (IOException e) {
+      throw new DiskErrorException("Directory is not writable: "
+                                   + dir.toString(), e);
+    }
+
+    // Make sure the directory is executable by trying to cd into it.  This
+    // launches a separate process.  It does not change the working directory of
+    // the current process.
+    try {
+      String[] cdCmd = new String[] { "cmd", "/C", "cd",
+          dir.getAbsolutePath() };
+      Shell.execCommand(null, cdCmd, SHELL_TIMEOUT);
+    } catch (Shell.ExitCodeException e) {
+      throw new DiskErrorException("Directory is not executable: "
+                                   + dir.toString(), e);
+    } catch (IOException e) {
+      throw new DiskErrorException("Directory is not executable: "
+                                   + dir.toString(), e);
+    }
   }
 }

+ 20 - 23
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java

@@ -25,10 +25,13 @@ import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
 import static org.apache.hadoop.test.MockitoMaker.*;
-import org.apache.hadoop.fs.permission.FsPermission;
+import static org.apache.hadoop.fs.permission.FsAction.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.Shell;
 
@@ -110,29 +113,21 @@ public class TestDiskChecker {
 
   private void _checkDirs(boolean isDir, FsPermission perm, boolean success)
       throws Throwable {
-    File localDir = make(stub(File.class).returning(true).from.exists());
-    when(localDir.mkdir()).thenReturn(true);
-    Path dir = mock(Path.class);
-    LocalFileSystem fs = make(stub(LocalFileSystem.class)
-        .returning(localDir).from.pathToFile(dir));
-    FileStatus stat = make(stub(FileStatus.class)
-        .returning(perm).from.getPermission());
-    when(stat.isDirectory()).thenReturn(isDir);
-    when(fs.getFileStatus(dir)).thenReturn(stat);
-
+    File localDir = File.createTempFile("test", "tmp");
+    if (isDir) {
+      localDir.delete();
+      localDir.mkdir();
+    }
+    Shell.execCommand(Shell.getSetPermissionCommand(String.format("%04o",
+      perm.toShort()), false, localDir.getAbsolutePath()));
     try {
-      DiskChecker.checkDir(fs, dir, perm);
-
-      verify(stat).isDirectory();
-      verify(fs, times(2)).getFileStatus(dir);
-      verify(stat, times(2)).getPermission();
+      DiskChecker.checkDir(FileSystem.getLocal(new Configuration()),
+        new Path(localDir.getAbsolutePath()), perm);
       assertTrue("checkDir success", success);
-    }
-    catch (DiskErrorException e) {
+    } catch (DiskErrorException e) {
       assertFalse("checkDir success", success);
-      e.printStackTrace();
     }
-    System.out.println("checkDir success: "+ success);
+    localDir.delete();
   }
 
   /**
@@ -168,8 +163,10 @@ public class TestDiskChecker {
   private void _checkDirs(boolean isDir, String perm, boolean success)
       throws Throwable {
     File localDir = File.createTempFile("test", "tmp");
-    localDir.delete();
-    localDir.mkdir();
+    if (isDir) {
+      localDir.delete();
+      localDir.mkdir();
+    }
     Shell.execCommand(Shell.getSetPermissionCommand(perm, false,
                                                     localDir.getAbsolutePath()));
     try {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/Server.java

@@ -232,7 +232,7 @@ public class Server {
    * path.
    */
   private String checkAbsolutePath(String value, String name) {
-    if (!value.startsWith("/")) {
+    if (!new File(value).isAbsolute()) {
       throw new IllegalArgumentException(
         MessageFormat.format("[{0}] must be an absolute path [{1}]", name, value));
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -364,7 +364,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     }
   }
 
-  private void testSetPermission() throws Exception {
+  protected void testSetPermission() throws Exception {
     FileSystem fs = FileSystem.get(getProxiedFSConf());
     Path path = new Path(getProxiedFSTestDir(), "foodir");
     fs.mkdirs(path);

+ 30 - 15
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java

@@ -20,8 +20,13 @@ package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.test.TestDirHelper;
+import org.junit.Assert;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
@@ -65,21 +70,31 @@ public class TestHttpFSFileSystemLocalFileSystem extends BaseTestHttpFSWith {
   }
 
   protected Path addPrefix(Path path) {
-    URI uri = path.toUri();
-    try {
-      if (uri.getAuthority() != null) {
-        uri = new URI(uri.getScheme(),
-                      uri.getAuthority(), PATH_PREFIX + uri.getPath());
-      }
-      else {
-        if (uri.getPath().startsWith("/")) {
-          uri = new URI(PATH_PREFIX + uri.getPath());
-        }
-      }
-    } catch (URISyntaxException ex) {
-      throw new RuntimeException("It should not happen: " + ex.toString(), ex);
-    }
-    return new Path(uri);
+    return Path.mergePaths(new Path(PATH_PREFIX), path);
   }
 
+  @Override
+  protected void testSetPermission() throws Exception {
+    if (Path.WINDOWS) {
+      FileSystem fs = FileSystem.get(getProxiedFSConf());
+      Path path = new Path(getProxiedFSTestDir(), "foodir");
+      fs.mkdirs(path);
+
+      fs = getHttpFSFileSystem();
+      FsPermission permission1 = new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE);
+      fs.setPermission(path, permission1);
+      fs.close();
+
+      fs = FileSystem.get(getProxiedFSConf());
+      FileStatus status1 = fs.getFileStatus(path);
+      fs.close();
+      FsPermission permission2 = status1.getPermission();
+      Assert.assertEquals(permission2, permission1);
+
+      // sticky bit not supported on Windows with local file system, so the
+      // subclass skips that part of the test
+    } else {
+      super.testSetPermission();
+    }
+  }
 }

+ 24 - 10
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.test.HTestCase;
 import org.apache.hadoop.test.TestDir;
 import org.apache.hadoop.test.TestDirHelper;
 import org.apache.hadoop.test.TestException;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 
@@ -50,21 +51,24 @@ public class TestServer extends HTestCase {
   @Test
   @TestDir
   public void constructorsGetters() throws Exception {
-    Server server = new Server("server", "/a", "/b", "/c", "/d", new Configuration(false));
-    assertEquals(server.getHomeDir(), "/a");
-    assertEquals(server.getConfigDir(), "/b");
-    assertEquals(server.getLogDir(), "/c");
-    assertEquals(server.getTempDir(), "/d");
+    Server server = new Server("server", getAbsolutePath("/a"),
+      getAbsolutePath("/b"), getAbsolutePath("/c"), getAbsolutePath("/d"),
+      new Configuration(false));
+    assertEquals(server.getHomeDir(), getAbsolutePath("/a"));
+    assertEquals(server.getConfigDir(), getAbsolutePath("/b"));
+    assertEquals(server.getLogDir(), getAbsolutePath("/c"));
+    assertEquals(server.getTempDir(), getAbsolutePath("/d"));
     assertEquals(server.getName(), "server");
     assertEquals(server.getPrefix(), "server");
     assertEquals(server.getPrefixedName("name"), "server.name");
     assertNotNull(server.getConfig());
 
-    server = new Server("server", "/a", "/b", "/c", "/d");
-    assertEquals(server.getHomeDir(), "/a");
-    assertEquals(server.getConfigDir(), "/b");
-    assertEquals(server.getLogDir(), "/c");
-    assertEquals(server.getTempDir(), "/d");
+    server = new Server("server", getAbsolutePath("/a"), getAbsolutePath("/b"),
+      getAbsolutePath("/c"), getAbsolutePath("/d"));
+    assertEquals(server.getHomeDir(), getAbsolutePath("/a"));
+    assertEquals(server.getConfigDir(), getAbsolutePath("/b"));
+    assertEquals(server.getLogDir(), getAbsolutePath("/c"));
+    assertEquals(server.getTempDir(), getAbsolutePath("/d"));
     assertEquals(server.getName(), "server");
     assertEquals(server.getPrefix(), "server");
     assertEquals(server.getPrefixedName("name"), "server.name");
@@ -793,4 +797,14 @@ public class TestServer extends HTestCase {
     server.destroy();
   }
 
+  /**
+   * Creates an absolute path by appending the given relative path to the test
+   * root.
+   * 
+   * @param relativePath String relative path
+   * @return String absolute path formed by appending relative path to test root
+   */
+  private static String getAbsolutePath(String relativePath) {
+    return new File(TestDirHelper.getTestDir(), relativePath).getAbsolutePath();
+  }
 }

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestHostnameFilter.java

@@ -50,7 +50,10 @@ public class TestHostnameFilter extends HTestCase {
       @Override
       public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse)
         throws IOException, ServletException {
-        assertTrue(HostnameFilter.get().contains("localhost"));
+        // Hostname was set to "localhost", but may get resolved automatically to
+        // "127.0.0.1" depending on OS.
+        assertTrue(HostnameFilter.get().contains("localhost") ||
+          HostnameFilter.get().contains("127.0.0.1"));
         invoked.set(true);
       }
     };

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java

@@ -63,7 +63,7 @@ public class TestDirHelper implements MethodRule {
   static {
     try {
       TEST_DIR_ROOT = System.getProperty(TEST_DIR_PROP, new File("target").getAbsolutePath());
-      if (!TEST_DIR_ROOT.startsWith("/")) {
+      if (!new File(TEST_DIR_ROOT).isAbsolute()) {
         System.err.println(MessageFormat.format("System property [{0}]=[{1}] must be set to an absolute path",
                                                 TEST_DIR_PROP, TEST_DIR_ROOT));
         System.exit(-1);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java

@@ -82,7 +82,8 @@ public class TestHdfsHelper extends TestDirHelper {
 
     private Path resetHdfsTestDir(Configuration conf) {
 
-      Path testDir = new Path("./" + TEST_DIR_ROOT, testName + "-" + counter.getAndIncrement());
+      Path testDir = new Path("/tmp/" + testName + "-" +
+        counter.getAndIncrement());
       try {
         // currentUser
         FileSystem fs = FileSystem.get(conf);

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

@@ -255,7 +255,6 @@ Release 2.0.5-beta - UNRELEASED
 
     HDFS-4533. start-dfs.sh ignores additional parameters besides -upgrade.
     (Fengdong Yu via suresh)
-
     HDFS-4765. Permission check of symlink deletion incorrectly throws
     UnresolvedLinkException. (Andrew Wang via atm)
 
@@ -333,7 +332,7 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4661. A few little code cleanups of some HDFS-347-related code. (Colin
     Patrick McCabe via atm)
 
-  BREAKDOWN OF HADOOP-8562 SUBTASKS
+  BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
     HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
     Bikas Saha, Lauren Yang, Chuan Liu, Thejas M Nair and Ivan Mitic via suresh)
@@ -347,6 +346,12 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4297. Fix issues related to datanode concurrent reading and writing on
     Windows. (Arpit Agarwal, Chuan Liu via suresh)
 
+    HDFS-4573. Fix TestINodeFile on Windows. (Arpit Agarwal via suresh)
+
+    HDFS-4572. Fix TestJournal failures on Windows. (Arpit Agarwal via suresh)
+
+    HDFS-4287. HTTPFS tests fail on Windows. (Chris Nauroth via suresh)
+
 Release 2.0.4-alpha - 2013-04-25
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java

@@ -219,8 +219,8 @@ class Journal implements Closeable {
   @Override // Closeable
   public void close() throws IOException {
     storage.close();
-    
     IOUtils.closeStream(committedTxnId);
+    IOUtils.closeStream(curSegment);
   }
   
   JNStorage getStorage() {

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -33,6 +33,7 @@ import java.util.Properties;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@@ -663,7 +664,9 @@ public abstract class Storage extends StorageInfo {
         file.write(jvmName.getBytes(Charsets.UTF_8));
         LOG.info("Lock on " + lockF + " acquired by nodename " + jvmName);
       } catch(OverlappingFileLockException oe) {
-        LOG.error("It appears that another namenode " + file.readLine() 
+        // Cannot read from the locked file on Windows.
+        String lockingJvmName = Path.WINDOWS ? "" : (" " + file.readLine());
+        LOG.error("It appears that another namenode" + lockingJvmName
             + " has already locked the storage directory");
         file.close();
         return null;

+ 15 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java

@@ -36,10 +36,7 @@ import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;
 import org.mockito.Mockito;
 
 public class TestJournal {
@@ -77,7 +74,7 @@ public class TestJournal {
     IOUtils.closeStream(journal);
   }
   
-  @Test
+  @Test (timeout = 10000)
   public void testEpochHandling() throws Exception {
     assertEquals(0, journal.getLastPromisedEpoch());
     NewEpochResponseProto newEpoch =
@@ -110,7 +107,7 @@ public class TestJournal {
     }
   }
   
-  @Test
+  @Test (timeout = 10000)
   public void testMaintainCommittedTxId() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     journal.startLogSegment(makeRI(1), 1);
@@ -125,7 +122,7 @@ public class TestJournal {
     assertEquals(3, journal.getCommittedTxnIdForTests());    
   }
   
-  @Test
+  @Test (timeout = 10000)
   public void testRestartJournal() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     journal.startLogSegment(makeRI(1), 1);
@@ -149,7 +146,7 @@ public class TestJournal {
     assertEquals(1, newEpoch.getLastSegmentTxId());
   }
   
-  @Test
+  @Test (timeout = 10000)
   public void testFormatResetsCachedValues() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 12345L);
     journal.startLogSegment(new RequestInfo(JID, 12345L, 1L, 0L), 1L);
@@ -158,6 +155,8 @@ public class TestJournal {
     assertEquals(12345L, journal.getLastWriterEpoch());
     assertTrue(journal.isFormatted());
     
+    // Close the journal in preparation for reformatting it.
+    journal.close();
     journal.format(FAKE_NSINFO_2);
     
     assertEquals(0, journal.getLastPromisedEpoch());
@@ -170,7 +169,7 @@ public class TestJournal {
    * before any transactions are written, that the next newEpoch() call
    * returns the prior segment txid as its most recent segment.
    */
-  @Test
+  @Test (timeout = 10000)
   public void testNewEpochAtBeginningOfSegment() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     journal.startLogSegment(makeRI(1), 1);
@@ -182,7 +181,7 @@ public class TestJournal {
     assertEquals(1, resp.getLastSegmentTxId());
   }
   
-  @Test
+  @Test (timeout = 10000)
   public void testJournalLocking() throws Exception {
     Assume.assumeTrue(journal.getStorage().getStorageDir(0).isLockSupported());
     StorageDirectory sd = journal.getStorage().getStorageDir(0);
@@ -206,13 +205,14 @@ public class TestJournal {
     // Hence, should be able to create a new Journal in the same dir.
     Journal journal2 = new Journal(TEST_LOG_DIR, JID, mockErrorReporter);
     journal2.newEpoch(FAKE_NSINFO, 2);
+    journal2.close();
   }
   
   /**
    * Test finalizing a segment after some batch of edits were missed.
    * This should fail, since we validate the log before finalization.
    */
-  @Test
+  @Test (timeout = 10000)
   public void testFinalizeWhenEditsAreMissed() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     journal.startLogSegment(makeRI(1), 1);
@@ -246,7 +246,7 @@ public class TestJournal {
    * Ensure that finalizing a segment which doesn't exist throws the
    * appropriate exception.
    */
-  @Test
+  @Test (timeout = 10000)
   public void testFinalizeMissingSegment() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     try {
@@ -267,7 +267,7 @@ public class TestJournal {
    * Eventually, the connection comes back, and the NN tries to start a new
    * segment at a higher txid. This should abort the old one and succeed.
    */
-  @Test
+  @Test (timeout = 10000)
   public void testAbortOldSegmentIfFinalizeIsMissed() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     
@@ -296,7 +296,7 @@ public class TestJournal {
    * Test behavior of startLogSegment() when a segment with the
    * same transaction ID already exists.
    */
-  @Test
+  @Test (timeout = 10000)
   public void testStartLogSegmentWhenAlreadyExists() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
     
@@ -345,7 +345,7 @@ public class TestJournal {
     return new RequestInfo(JID, 1, serial, 0);
   }
   
-  @Test
+  @Test (timeout = 10000)
   public void testNamespaceVerification() throws Exception {
     journal.newEpoch(FAKE_NSINFO, 1);
 

+ 35 - 28
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -197,34 +197,41 @@ public class TestINodeFile {
     long fileLen = 1024;
     replication = 3;
     Configuration conf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
-        replication).build();
-    cluster.waitActive();
-    FSNamesystem fsn = cluster.getNamesystem();
-    FSDirectory fsdir = fsn.getFSDirectory();
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    
-    // Create a file for test
-    final Path dir = new Path("/dir");
-    final Path file = new Path(dir, "file");
-    DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
-    
-    // Check the full path name of the INode associating with the file
-    INode fnode = fsdir.getINode(file.toString());
-    assertEquals(file.toString(), fnode.getFullPathName());
-    
-    // Call FSDirectory#unprotectedSetQuota which calls
-    // INodeDirectory#replaceChild
-    dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
-    final Path newDir = new Path("/newdir");
-    final Path newFile = new Path(newDir, "file");
-    // Also rename dir
-    dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
-    // /dir/file now should be renamed to /newdir/file
-    fnode = fsdir.getINode(newFile.toString());
-    // getFullPathName can return correct result only if the parent field of
-    // child node is set correctly
-    assertEquals(newFile.toString(), fnode.getFullPathName());
+    MiniDFSCluster cluster = null;
+    try {
+      cluster =
+          new MiniDFSCluster.Builder(conf).numDataNodes(replication).build();
+      cluster.waitActive();
+      FSNamesystem fsn = cluster.getNamesystem();
+      FSDirectory fsdir = fsn.getFSDirectory();
+      DistributedFileSystem dfs = cluster.getFileSystem();
+
+      // Create a file for test
+      final Path dir = new Path("/dir");
+      final Path file = new Path(dir, "file");
+      DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
+
+      // Check the full path name of the INode associating with the file
+      INode fnode = fsdir.getINode(file.toString());
+      assertEquals(file.toString(), fnode.getFullPathName());
+
+      // Call FSDirectory#unprotectedSetQuota which calls
+      // INodeDirectory#replaceChild
+      dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
+      final Path newDir = new Path("/newdir");
+      final Path newFile = new Path(newDir, "file");
+      // Also rename dir
+      dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
+      // /dir/file now should be renamed to /newdir/file
+      fnode = fsdir.getINode(newFile.toString());
+      // getFullPathName can return correct result only if the parent field of
+      // child node is set correctly
+      assertEquals(newFile.toString(), fnode.getFullPathName());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
   }
   
   @Test