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
   $ mvn install
 
-Then, generate ecplise project files.
+Then, generate eclipse project files.
 
   $ mvn eclipse:eclipse -DskipTests
 
@@ -147,10 +147,10 @@ Requirements:
 * Windows System
 * JDK 1.6
 * 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)
+* Findbugs 1.3.9 (if running findbugs)
 * 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)
 
 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
 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:
 
-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)
 
     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
 
   INCOMPATIBLE CHANGES

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

@@ -557,7 +557,9 @@
     <profile>
       <id>native-win</id>
       <activation>
-        <activeByDefault>false</activeByDefault>
+        <os>
+          <family>Windows</family>
+        </os>
       </activation>
       <build>
         <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;
 
-  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 {
     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>();
   }
 
-  protected OSType getOSType() {
-    return OS_TYPE;
-  }
-  
   /// ACCESSORS
 
   /** @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. */
   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. */
@@ -138,16 +108,23 @@ public class DF extends Shell {
       throw new FileNotFoundException("Specified path " + dirFile.getPath()
           + "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;
   }
   
@@ -163,24 +140,16 @@ public class DF extends Shell {
       mount;
   }
 
-  @Override
-  protected void run() throws IOException {
-    if (WINDOWS) {
-      try {
-        this.mount = dirFile.getCanonicalPath().substring(0,2);
-      } catch (IOException e) {
-      }
-      return;
-    }
-    super.run();
-  }
-
   @Override
   protected String[] getExecString() {
     // 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"};
+    }
   }
 
   @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 org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
 import org.junit.Test;
 import static org.junit.Assert.*;
 
 public class TestDFVariations {
 
   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);
-      this.osName = osName;
-    }
-    @Override
-    public DF.OSType getOSType() {
-      return DF.getOSType(osName);
     }
+
     @Override
     protected String[] getExecString() {
       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)
   public void testDFInvalidPath() throws Exception {
     // 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
     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
 
   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.Random;
 import java.util.Scanner;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.GZIPOutputStream;
 
@@ -68,7 +69,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERV
  */
 public class TestDFSShell {
   private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
-  
+  private static AtomicInteger counter = new AtomicInteger();
+
   static final String TEST_ROOT_DIR =
     new Path(System.getProperty("test.build.data","/tmp"))
     .toString().replace(' ', '+');
@@ -512,7 +514,7 @@ public class TestDFSShell {
       createLocalFile(furi);
       argv = new String[3];
       argv[0] = "-put";
-      argv[1] = furi.toString();
+      argv[1] = furi.toURI().toString();
       argv[2] = dstFs.getUri().toString() + "/furi";
       ret = ToolRunner.run(shell, argv);
       assertEquals(" put is working ", 0, ret);
@@ -867,52 +869,59 @@ public class TestDFSShell {
     shell.setConf(conf);
     
     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);
-     
-     //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 {
       try {
         fs.close();
@@ -1595,27 +1604,29 @@ public class TestDFSShell {
   // force Copy Option is -f
   @Test (timeout = 30000)
   public void testCopyCommandsWithForceOption() throws Exception {
+    final int SUCCESS = 0;
+    final int ERROR = 1;
+
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();
     FsShell shell = null;
     FileSystem fs = null;
     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);
     try {
       fs = cluster.getFileSystem();
       fs.mkdirs(hdfsTestDir);
       localFile.createNewFile();
-      writeFile(fs, new Path(TEST_ROOT_DIR, "testFileForPut"));
+      writeFile(fs, new Path(testdir, "testFileForPut"));
       shell = new FsShell();
 
       // Tests for put
       String[] argv = new String[] { "-put", "-f", localfilepath, testdir };
       int res = ToolRunner.run(shell, argv);
-      int SUCCESS = 0;
-      int ERROR = 1;
       assertEquals("put -f is not working", SUCCESS, res);
 
       argv = new String[] { "-put", localfilepath, testdir };
@@ -1687,8 +1698,13 @@ public class TestDFSShell {
     try {
       // Create and delete a file
       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;
       String[] argv = new String[] { "-rm", testFile };
       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;
 
 import static org.junit.Assert.*;
+import static org.junit.Assume.*;
 
 import java.io.File;
 import java.io.IOException;
@@ -43,7 +44,7 @@ import org.junit.Test;
 
 public class TestNNWithQJM {
   Configuration conf = new HdfsConfiguration();
-  private MiniJournalCluster mjc;
+  private MiniJournalCluster mjc = null;
   private Path TEST_PATH = 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 {
     if (mjc != null) {
       mjc.shutdown();
+      mjc = null;
     }
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testLogAndRestart() throws IOException {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
@@ -93,9 +95,12 @@ public class TestNNWithQJM {
       cluster.shutdown();
     }
   }
-  
-  @Test
+
+  @Test (timeout = 30000)
   public void testNewNamenodeTakesOverWriter() throws Exception {
+    // Skip the test on Windows. See HDFS-4584.
+    assumeTrue(!Path.WINDOWS);
+
     File nn1Dir = new File(
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
     File nn2Dir = new File(
@@ -154,7 +159,7 @@ public class TestNNWithQJM {
     }
   }
 
-  @Test
+  @Test (timeout = 30000)
   public void testMismatchedNNIsRejected() throws Exception {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
@@ -188,8 +193,8 @@ public class TestNNWithQJM {
           "Unable to start log segment 1: too few journals", ioe);
     }
   }
-  
-  @Test
+
+  @Test (timeout = 30000)
   public void testWebPageHasQjmInfo() throws Exception {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");