Browse Source

HADOOP-8562. Merge r1459586 for HDFS-4615, r1459592 for HDFS-4584, r1459643, r1459642 for HADOOP-9387, r1460086 for HADOOP-9353

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1485932 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
0a96ae767f

+ 8 - 18
BUILDING.txt

@@ -107,7 +107,7 @@ When you import the project to eclipse, install hadoop-maven-plugins at first.
   $ cd hadoop-maven-plugins
   $ cd hadoop-maven-plugins
   $ mvn install
   $ mvn install
 
 
-Then, generate ecplise project files.
+Then, generate eclipse project files.
 
 
   $ mvn eclipse:eclipse -DskipTests
   $ mvn eclipse:eclipse -DskipTests
 
 
@@ -147,10 +147,10 @@ Requirements:
 * Windows System
 * Windows System
 * JDK 1.6
 * JDK 1.6
 * Maven 3.0
 * Maven 3.0
-* Findbugs 1.3.9 (if running findbugs)
+* Windows SDK or Visual Studio 2010 Professional
 * ProtocolBuffer 2.4.1+ (for MapReduce and HDFS)
 * ProtocolBuffer 2.4.1+ (for MapReduce and HDFS)
+* Findbugs 1.3.9 (if running findbugs)
 * Unix command-line tools from GnuWin32 or Cygwin: sh, mkdir, rm, cp, tar, gzip
 * Unix command-line tools from GnuWin32 or Cygwin: sh, mkdir, rm, cp, tar, gzip
-* Windows SDK or Visual Studio 2010 Professional
 * Internet connection for first build (to fetch all Maven and Hadoop dependencies)
 * Internet connection for first build (to fetch all Maven and Hadoop dependencies)
 
 
 If using Visual Studio, it must be Visual Studio 2010 Professional (not 2012).
 If using Visual Studio, it must be Visual Studio 2010 Professional (not 2012).
@@ -185,23 +185,13 @@ set Platform=Win32 (when building on a 32-bit system)
 Several tests require that the user must have the Create Symbolic Links
 Several tests require that the user must have the Create Symbolic Links
 privilege.
 privilege.
 
 
-All Maven goals are the same as described above, with the addition of profile
--Pnative-win to trigger building Windows native components.  The native
-components are required (not optional) on Windows.  For example:
-
- * Run tests                 : mvn -Pnative-win test
+All Maven goals are the same as described above with the exception that
+native code is built by enabling the 'native-win' Maven profile. -Pnative-win 
+is enabled by default when building on Windows since the native components 
+are required (not optional) on Windows.
 
 
 ----------------------------------------------------------------------------------
 ----------------------------------------------------------------------------------
 Building distributions:
 Building distributions:
 
 
-Create binary distribution with native code and with documentation:
-
-  $ mvn package -Pdist,native-win,docs -DskipTests -Dtar
-
-Create source distribution:
-
-  $ mvn package -Pnative-win,src -DskipTests
-
-Create source and binary distributions with native code and documentation:
+ * Build distribution with native code    : mvn package [-Pdist][-Pdocs][-Psrc][-Dtar]
 
 
-  $ mvn package -Pdist,native-win,docs,src -DskipTests -Dtar

+ 7 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -319,7 +319,13 @@ Release 2.0.5-beta - UNRELEASED
     Windows with NTFS ACLs. (Chris Nauroth via suresh)
     Windows with NTFS ACLs. (Chris Nauroth via suresh)
 
 
     HADOOP-9388. TestFsShellCopy fails on Windows. (Ivan Mitic via suresh)
     HADOOP-9388. TestFsShellCopy fails on Windows. (Ivan Mitic via suresh)
-    
+
+    HADOOP-9387. Fix DF so that it won't execute a shell command on Windows
+    to compute the file system/mount point.  (Ivan Mitic via szetszwo)
+
+    HADOOP-9353. Activate native-win maven profile by default on Windows.
+    (Arpit Agarwal via szetszwo)
+
 Release 2.0.4-beta - UNRELEASED
 Release 2.0.4-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 3 - 1
hadoop-common-project/hadoop-common/pom.xml

@@ -557,7 +557,9 @@
     <profile>
     <profile>
       <id>native-win</id>
       <id>native-win</id>
       <activation>
       <activation>
-        <activeByDefault>false</activeByDefault>
+        <os>
+          <family>Windows</family>
+        </os>
       </activation>
       </activation>
       <build>
       <build>
         <plugins>
         <plugins>

+ 28 - 59
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java

@@ -50,37 +50,6 @@ public class DF extends Shell {
   
   
   private ArrayList<String> output;
   private ArrayList<String> output;
 
 
-  enum OSType {
-    OS_TYPE_UNIX("UNIX"),
-    OS_TYPE_WIN("Windows"),
-    OS_TYPE_SOLARIS("SunOS"),
-    OS_TYPE_MAC("Mac"),
-    OS_TYPE_AIX("AIX");
-
-    private String id;
-    OSType(String id) {
-      this.id = id;
-    }
-    public boolean match(String osStr) {
-      return osStr != null && osStr.indexOf(id) >= 0;
-    }
-    String getId() {
-      return id;
-    }
-  }
-
-  private static final String OS_NAME = System.getProperty("os.name");
-  private static final OSType OS_TYPE = getOSType(OS_NAME);
-
-  protected static OSType getOSType(String osName) {
-    for (OSType ost : EnumSet.allOf(OSType.class)) {
-      if (ost.match(osName)) {
-        return ost;
-      }
-    }
-    return OSType.OS_TYPE_UNIX;
-  }
-
   public DF(File path, Configuration conf) throws IOException {
   public DF(File path, Configuration conf) throws IOException {
     this(path, conf.getLong(CommonConfigurationKeys.FS_DF_INTERVAL_KEY, DF.DF_INTERVAL_DEFAULT));
     this(path, conf.getLong(CommonConfigurationKeys.FS_DF_INTERVAL_KEY, DF.DF_INTERVAL_DEFAULT));
   }
   }
@@ -92,10 +61,6 @@ public class DF extends Shell {
     this.output = new ArrayList<String>();
     this.output = new ArrayList<String>();
   }
   }
 
 
-  protected OSType getOSType() {
-    return OS_TYPE;
-  }
-  
   /// ACCESSORS
   /// ACCESSORS
 
 
   /** @return the canonical path to the volume we're checking. */
   /** @return the canonical path to the volume we're checking. */
@@ -105,8 +70,13 @@ public class DF extends Shell {
 
 
   /** @return a string indicating which filesystem volume we're checking. */
   /** @return a string indicating which filesystem volume we're checking. */
   public String getFilesystem() throws IOException {
   public String getFilesystem() throws IOException {
-    run();
-    return filesystem;
+    if (Shell.WINDOWS) {
+      this.filesystem = dirFile.getCanonicalPath().substring(0, 2);
+      return this.filesystem;
+    } else {
+      run();
+      return filesystem;
+    }
   }
   }
 
 
   /** @return the capacity of the measured filesystem in bytes. */
   /** @return the capacity of the measured filesystem in bytes. */
@@ -138,16 +108,23 @@ public class DF extends Shell {
       throw new FileNotFoundException("Specified path " + dirFile.getPath()
       throw new FileNotFoundException("Specified path " + dirFile.getPath()
           + "does not exist");
           + "does not exist");
     }
     }
-    run();
-    // Skip parsing if df was not successful
-    if (getExitCode() != 0) {
-      StringBuffer sb = new StringBuffer("df could not be run successfully: ");
-      for (String line: output) {
-        sb.append(line);
+
+    if (Shell.WINDOWS) {
+      // Assume a drive letter for a mount point
+      this.mount = dirFile.getCanonicalPath().substring(0, 2);
+    } else {
+      run();
+      // Skip parsing if df was not successful
+      if (getExitCode() != 0) {
+        StringBuffer sb = new StringBuffer("df could not be run successfully: ");
+        for (String line: output) {
+          sb.append(line);
+        }
+        throw new IOException(sb.toString());
       }
       }
-      throw new IOException(sb.toString());
+      parseOutput();
     }
     }
-    parseOutput();
+
     return mount;
     return mount;
   }
   }
   
   
@@ -163,24 +140,16 @@ public class DF extends Shell {
       mount;
       mount;
   }
   }
 
 
-  @Override
-  protected void run() throws IOException {
-    if (WINDOWS) {
-      try {
-        this.mount = dirFile.getCanonicalPath().substring(0,2);
-      } catch (IOException e) {
-      }
-      return;
-    }
-    super.run();
-  }
-
   @Override
   @Override
   protected String[] getExecString() {
   protected String[] getExecString() {
     // ignoring the error since the exit code it enough
     // ignoring the error since the exit code it enough
-    return (WINDOWS)? new String[]{"cmd", "/c", "df -k " + dirPath + " 2>nul"}:
-        new String[] {"bash","-c","exec 'df' '-k' '-P' '" + dirPath 
+    if (Shell.WINDOWS){
+      throw new AssertionError(
+          "DF.getExecString() should never be called on Windows");
+    } else {
+      return new String[] {"bash","-c","exec 'df' '-k' '-P' '" + dirPath 
                       + "' 2>/dev/null"};
                       + "' 2>/dev/null"};
+    }
   }
   }
 
 
   @Override
   @Override

+ 16 - 14
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFVariations.java

@@ -29,21 +29,17 @@ import java.util.EnumSet;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
 import org.junit.Test;
 import org.junit.Test;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 public class TestDFVariations {
 public class TestDFVariations {
 
 
   public static class XXDF extends DF {
   public static class XXDF extends DF {
-    private final String osName;
-    public XXDF(String osName) throws IOException {
+    public XXDF() throws IOException {
       super(new File(System.getProperty("test.build.data","/tmp")), 0L);
       super(new File(System.getProperty("test.build.data","/tmp")), 0L);
-      this.osName = osName;
-    }
-    @Override
-    public DF.OSType getOSType() {
-      return DF.getOSType(osName);
     }
     }
+
     @Override
     @Override
     protected String[] getExecString() {
     protected String[] getExecString() {
       return new String[] { "echo", "IGNORE\n", 
       return new String[] { "echo", "IGNORE\n", 
@@ -51,14 +47,20 @@ public class TestDFVariations {
     }
     }
   }
   }
 
 
-  @Test(timeout=5000)
-  public void testOSParsing() throws Exception {
-    for (DF.OSType ost : EnumSet.allOf(DF.OSType.class)) {
-      XXDF df = new XXDF(ost.getId());
-      assertEquals(ost.getId() + " mount", "/foo/bar", df.getMount());
-    }
+  public void testMountAndFileSystem() throws Exception {
+    XXDF df = new XXDF();
+    String expectedMount =
+        Shell.WINDOWS ? df.getDirPath().substring(0, 2) : "/foo/bar";
+    String expectedFileSystem =
+        Shell.WINDOWS ? df.getDirPath().substring(0, 2) : "/dev/sda3";
+
+    assertEquals("Invalid mount point",
+        expectedMount, df.getMount());
+
+    assertEquals("Invalid filesystem",
+        expectedFileSystem, df.getFilesystem());
   }
   }
-  
+
   @Test(timeout=5000)
   @Test(timeout=5000)
   public void testDFInvalidPath() throws Exception {
   public void testDFInvalidPath() throws Exception {
     // Generate a path that doesn't exist
     // Generate a path that doesn't exist

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

@@ -363,6 +363,12 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4607.  In TestGetConf.testGetSpecificKey(), use a platform-specific
     HDFS-4607.  In TestGetConf.testGetSpecificKey(), use a platform-specific
     line separator; otherwise, it fails on Windows.  (Ivan Mitic via szetszwo)
     line separator; otherwise, it fails on Windows.  (Ivan Mitic via szetszwo)
 
 
+    HDFS-4615. Fix TestDFSShell failures on Windows.  (Arpit Agarwal
+    via szetszwo)
+
+    HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
+    (Arpit Agarwal via szetszwo)
+
 Release 2.0.4-alpha - 2013-04-25
 Release 2.0.4-alpha - 2013-04-25
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 69 - 53
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -35,6 +35,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 import java.util.Random;
 import java.util.Scanner;
 import java.util.Scanner;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.GZIPOutputStream;
 import java.util.zip.GZIPOutputStream;
 
 
@@ -68,7 +69,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERV
  */
  */
 public class TestDFSShell {
 public class TestDFSShell {
   private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
   private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
-  
+  private static AtomicInteger counter = new AtomicInteger();
+
   static final String TEST_ROOT_DIR =
   static final String TEST_ROOT_DIR =
     new Path(System.getProperty("test.build.data","/tmp"))
     new Path(System.getProperty("test.build.data","/tmp"))
     .toString().replace(' ', '+');
     .toString().replace(' ', '+');
@@ -512,7 +514,7 @@ public class TestDFSShell {
       createLocalFile(furi);
       createLocalFile(furi);
       argv = new String[3];
       argv = new String[3];
       argv[0] = "-put";
       argv[0] = "-put";
-      argv[1] = furi.toString();
+      argv[1] = furi.toURI().toString();
       argv[2] = dstFs.getUri().toString() + "/furi";
       argv[2] = dstFs.getUri().toString() + "/furi";
       ret = ToolRunner.run(shell, argv);
       ret = ToolRunner.run(shell, argv);
       assertEquals(" put is working ", 0, ret);
       assertEquals(" put is working ", 0, ret);
@@ -867,52 +869,59 @@ public class TestDFSShell {
     shell.setConf(conf);
     shell.setConf(conf);
     
     
     try {
     try {
-     //first make dir
-     Path dir = new Path(chmodDir);
-     fs.delete(dir, true);
-     fs.mkdirs(dir);
+      //first make dir
+      Path dir = new Path(chmodDir);
+      fs.delete(dir, true);
+      fs.mkdirs(dir);
 
 
-     confirmPermissionChange(/* Setting */ "u+rwx,g=rw,o-rwx",
+      confirmPermissionChange(/* Setting */ "u+rwx,g=rw,o-rwx",
                              /* Should give */ "rwxrw----", fs, shell, dir);
                              /* Should give */ "rwxrw----", fs, shell, dir);
-     
-     //create an empty file
-     Path file = new Path(chmodDir, "file");
-     TestDFSShell.writeFile(fs, file);
-
-     //test octal mode
-     confirmPermissionChange( "644", "rw-r--r--", fs, shell, file);
-
-     //test recursive
-     runCmd(shell, "-chmod", "-R", "a+rwX", chmodDir);
-     assertEquals("rwxrwxrwx",
-                  fs.getFileStatus(dir).getPermission().toString()); 
-     assertEquals("rw-rw-rw-",
-                  fs.getFileStatus(file).getPermission().toString());
-
-     // test sticky bit on directories
-     Path dir2 = new Path(dir, "stickybit" );
-     fs.mkdirs(dir2 );
-     LOG.info("Testing sticky bit on: " + dir2);
-     LOG.info("Sticky bit directory initial mode: " + 
-                   fs.getFileStatus(dir2).getPermission());
-     
-     confirmPermissionChange("u=rwx,g=rx,o=rx", "rwxr-xr-x", fs, shell, dir2);
-     
-     confirmPermissionChange("+t", "rwxr-xr-t", fs, shell, dir2);
-
-     confirmPermissionChange("-t", "rwxr-xr-x", fs, shell, dir2);
-
-     confirmPermissionChange("=t", "--------T", fs, shell, dir2);
-
-     confirmPermissionChange("0000", "---------", fs, shell, dir2);
-
-     confirmPermissionChange("1666", "rw-rw-rwT", fs, shell, dir2);
-
-     confirmPermissionChange("777", "rwxrwxrwt", fs, shell, dir2);
-     
-     fs.delete(dir2, true);
-     fs.delete(dir, true);
-     
+
+      //create an empty file
+      Path file = new Path(chmodDir, "file");
+      TestDFSShell.writeFile(fs, file);
+
+      //test octal mode
+      confirmPermissionChange("644", "rw-r--r--", fs, shell, file);
+
+      //test recursive
+      runCmd(shell, "-chmod", "-R", "a+rwX", chmodDir);
+      assertEquals("rwxrwxrwx",
+          fs.getFileStatus(dir).getPermission().toString());
+      assertEquals("rw-rw-rw-",
+          fs.getFileStatus(file).getPermission().toString());
+
+      // Skip "sticky bit" tests on Windows.
+      //
+      if (!Path.WINDOWS) {
+        // test sticky bit on directories
+        Path dir2 = new Path(dir, "stickybit");
+        fs.mkdirs(dir2);
+        LOG.info("Testing sticky bit on: " + dir2);
+        LOG.info("Sticky bit directory initial mode: " +
+            fs.getFileStatus(dir2).getPermission());
+
+        confirmPermissionChange("u=rwx,g=rx,o=rx", "rwxr-xr-x", fs, shell, dir2);
+
+        confirmPermissionChange("+t", "rwxr-xr-t", fs, shell, dir2);
+
+        confirmPermissionChange("-t", "rwxr-xr-x", fs, shell, dir2);
+
+        confirmPermissionChange("=t", "--------T", fs, shell, dir2);
+
+        confirmPermissionChange("0000", "---------", fs, shell, dir2);
+
+        confirmPermissionChange("1666", "rw-rw-rwT", fs, shell, dir2);
+
+        confirmPermissionChange("777", "rwxrwxrwt", fs, shell, dir2);
+
+        fs.delete(dir2, true);
+      } else {
+        LOG.info("Skipped sticky bit tests on Windows");
+      }
+
+      fs.delete(dir, true);
+
     } finally {
     } finally {
       try {
       try {
         fs.close();
         fs.close();
@@ -1595,27 +1604,29 @@ public class TestDFSShell {
   // force Copy Option is -f
   // force Copy Option is -f
   @Test (timeout = 30000)
   @Test (timeout = 30000)
   public void testCopyCommandsWithForceOption() throws Exception {
   public void testCopyCommandsWithForceOption() throws Exception {
+    final int SUCCESS = 0;
+    final int ERROR = 1;
+
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();
         .format(true).build();
     FsShell shell = null;
     FsShell shell = null;
     FileSystem fs = null;
     FileSystem fs = null;
     final File localFile = new File(TEST_ROOT_DIR, "testFileForPut");
     final File localFile = new File(TEST_ROOT_DIR, "testFileForPut");
-    final String localfilepath = localFile.getAbsolutePath();
-    final String testdir = TEST_ROOT_DIR + "/ForceTestDir";
+    final String localfilepath = new Path(localFile.getAbsolutePath()).toUri().toString();
+    final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithForceOption-"
+        + counter.getAndIncrement();
     final Path hdfsTestDir = new Path(testdir);
     final Path hdfsTestDir = new Path(testdir);
     try {
     try {
       fs = cluster.getFileSystem();
       fs = cluster.getFileSystem();
       fs.mkdirs(hdfsTestDir);
       fs.mkdirs(hdfsTestDir);
       localFile.createNewFile();
       localFile.createNewFile();
-      writeFile(fs, new Path(TEST_ROOT_DIR, "testFileForPut"));
+      writeFile(fs, new Path(testdir, "testFileForPut"));
       shell = new FsShell();
       shell = new FsShell();
 
 
       // Tests for put
       // Tests for put
       String[] argv = new String[] { "-put", "-f", localfilepath, testdir };
       String[] argv = new String[] { "-put", "-f", localfilepath, testdir };
       int res = ToolRunner.run(shell, argv);
       int res = ToolRunner.run(shell, argv);
-      int SUCCESS = 0;
-      int ERROR = 1;
       assertEquals("put -f is not working", SUCCESS, res);
       assertEquals("put -f is not working", SUCCESS, res);
 
 
       argv = new String[] { "-put", localfilepath, testdir };
       argv = new String[] { "-put", localfilepath, testdir };
@@ -1687,8 +1698,13 @@ public class TestDFSShell {
     try {
     try {
       // Create and delete a file
       // Create and delete a file
       fs = cluster.getFileSystem();
       fs = cluster.getFileSystem();
-      writeFile(fs, new Path(TEST_ROOT_DIR, "foo"));
-      final String testFile = TEST_ROOT_DIR + "/foo";
+
+      // Use a separate tmp dir for each invocation.
+      final String testdir = "/tmp/TestDFSShell-deleteFileUsingTrash-" +
+          counter.getAndIncrement();
+
+      writeFile(fs, new Path(testdir, "foo"));
+      final String testFile = testdir + "/foo";
       final String trashFile = shell.getCurrentTrashDir() + "/" + testFile;
       final String trashFile = shell.getCurrentTrashDir() + "/" + testFile;
       String[] argv = new String[] { "-rm", testFile };
       String[] argv = new String[] { "-rm", testFile };
       int res = ToolRunner.run(shell, argv);
       int res = ToolRunner.run(shell, argv);

+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.qjournal;
 package org.apache.hadoop.hdfs.qjournal;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
+import static org.junit.Assume.*;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
@@ -43,7 +44,7 @@ import org.junit.Test;
 
 
 public class TestNNWithQJM {
 public class TestNNWithQJM {
   Configuration conf = new HdfsConfiguration();
   Configuration conf = new HdfsConfiguration();
-  private MiniJournalCluster mjc;
+  private MiniJournalCluster mjc = null;
   private Path TEST_PATH = new Path("/test-dir");
   private Path TEST_PATH = new Path("/test-dir");
   private Path TEST_PATH_2 = new Path("/test-dir");
   private Path TEST_PATH_2 = new Path("/test-dir");
 
 
@@ -61,10 +62,11 @@ public class TestNNWithQJM {
   public void stopJNs() throws Exception {
   public void stopJNs() throws Exception {
     if (mjc != null) {
     if (mjc != null) {
       mjc.shutdown();
       mjc.shutdown();
+      mjc = null;
     }
     }
   }
   }
   
   
-  @Test
+  @Test (timeout = 30000)
   public void testLogAndRestart() throws IOException {
   public void testLogAndRestart() throws IOException {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
@@ -93,9 +95,12 @@ public class TestNNWithQJM {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
-  
-  @Test
+
+  @Test (timeout = 30000)
   public void testNewNamenodeTakesOverWriter() throws Exception {
   public void testNewNamenodeTakesOverWriter() throws Exception {
+    // Skip the test on Windows. See HDFS-4584.
+    assumeTrue(!Path.WINDOWS);
+
     File nn1Dir = new File(
     File nn1Dir = new File(
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
     File nn2Dir = new File(
     File nn2Dir = new File(
@@ -154,7 +159,7 @@ public class TestNNWithQJM {
     }
     }
   }
   }
 
 
-  @Test
+  @Test (timeout = 30000)
   public void testMismatchedNNIsRejected() throws Exception {
   public void testMismatchedNNIsRejected() throws Exception {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
@@ -188,8 +193,8 @@ public class TestNNWithQJM {
           "Unable to start log segment 1: too few journals", ioe);
           "Unable to start log segment 1: too few journals", ioe);
     }
     }
   }
   }
-  
-  @Test
+
+  @Test (timeout = 30000)
   public void testWebPageHasQjmInfo() throws Exception {
   public void testWebPageHasQjmInfo() throws Exception {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");