Explorar el Código

Merge r1490115 for HADOOP-8957, r1490118 for HADOOP-9131, r1490121 for HADOOP-9526, r1490125 for HADOOP-8982, r1490147 for HADOOP-8958 from branch-2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1490151 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas hace 12 años
padre
commit
f39ac7f332
Se han modificado 17 ficheros con 158 adiciones y 50 borrados
  1. 20 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 10 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
  3. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
  4. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java
  5. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFs.java
  6. 4 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
  7. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
  8. 3 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
  9. 19 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFs.java
  10. 6 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
  11. 6 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
  12. 21 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java
  13. 15 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java
  14. 6 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
  15. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  16. 16 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
  17. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

+ 20 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -215,6 +215,13 @@ Release 2.1.0-beta - UNRELEASED
     HADOOP-9593. stack trace printed at ERROR for all yarn clients without
     HADOOP-9593. stack trace printed at ERROR for all yarn clients without
     hadoop.home set (stevel)
     hadoop.home set (stevel)
 
 
+    HADOOP-8957 AbstractFileSystem#IsValidName should be overridden for
+    embedded file systems like ViewFs (Chris Nauroth via Sanjay Radia)
+
+    HADOOP-9607. Fixes in Javadoc build (Timothy St. Clair via cos)
+
+    HADOOP-9605. Update junit dependency. (Timothy St. Clair via cos)
+
   BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
   BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
 
     HADOOP-8924. Hadoop Common creating package-info.java must not depend on
     HADOOP-8924. Hadoop Common creating package-info.java must not depend on
@@ -389,9 +396,17 @@ Release 2.1.0-beta - UNRELEASED
 
 
     HADOOP-9397. Incremental dist tar build fails. (Chris Nauroth via jlowe)
     HADOOP-9397. Incremental dist tar build fails. (Chris Nauroth via jlowe)
 
 
-    HADOOP-9607. Fixes in Javadoc build (Timothy St. Clair via cos)
+    HADOOP-9131. Turn off TestLocalFileSystem#testListStatusWithColons on
+    Windows. (Chris Nauroth via suresh)
 
 
-    HADOOP-9605. Update junit dependency. (Timothy St. Clair via cos)
+    HADOOP-9526. TestShellCommandFencer and TestShell fail on Windows.
+    (Arpit Agarwal via suresh)
+
+    HADOOP-8982. TestSocketIOWithTimeout fails on Windows.
+    (Chris Nauroth via suresh)
+
+    HADOOP-8958. ViewFs:Non absolute mount name failures when running 
+    multiple tests on Windows. (Chris Nauroth via suresh)
 
 
 Release 2.0.5-alpha - UNRELEASED
 Release 2.0.5-alpha - UNRELEASED
 
 
@@ -732,6 +747,9 @@ Release 2.0.3-alpha - 2013-02-06
     HADOOP-9289. FsShell rm -f fails for non-matching globs. (Daryn Sharp via
     HADOOP-9289. FsShell rm -f fails for non-matching globs. (Daryn Sharp via
     suresh)
     suresh)
 
 
+    HADOOP-8958. ViewFs:Non absolute mount name failures when running 
+    multiple tests on Windows. (Chris Nauroth via suresh)
+
 Release 2.0.2-alpha - 2012-09-07 
 Release 2.0.2-alpha - 2012-09-07 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 10 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -85,14 +85,20 @@ public abstract class AbstractFileSystem {
   }
   }
   
   
   /**
   /**
-   * Prohibits names which contain a ".", "..", ":" or "/" 
+   * Returns true if the specified string is considered valid in the path part
+   * of a URI by this file system.  The default implementation enforces the rules
+   * of HDFS, but subclasses may override this method to implement specific
+   * validation rules for specific file systems.
+   * 
+   * @param src String source filename to check, path part of the URI
+   * @return boolean true if the specified string is considered valid
    */
    */
-  private static boolean isValidName(String src) {
-    // Check for ".." "." ":" "/"
+  public boolean isValidName(String src) {
+    // Prohibit ".." "." and anything containing ":"
     StringTokenizer tokens = new StringTokenizer(src, Path.SEPARATOR);
     StringTokenizer tokens = new StringTokenizer(src, Path.SEPARATOR);
     while(tokens.hasMoreTokens()) {
     while(tokens.hasMoreTokens()) {
       String element = tokens.nextToken();
       String element = tokens.nextToken();
-      if (element.equals("target/generated-sources") ||
+      if (element.equals("..") ||
           element.equals(".")  ||
           element.equals(".")  ||
           (element.indexOf(":") >= 0)) {
           (element.indexOf(":") >= 0)) {
         return false;
         return false;

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java

@@ -278,4 +278,9 @@ public abstract class FilterFs extends AbstractFileSystem {
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
     return myFs.getDelegationTokens(renewer);
     return myFs.getDelegationTokens(renewer);
   }
   }
+
+  @Override
+  public boolean isValidName(String src) {
+    return myFs.isValidName(src);
+  }
 }
 }

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java

@@ -158,6 +158,14 @@ public class RawLocalFs extends DelegateToFileSystem {
     }
     }
   }
   }
   
   
+   @Override
+   public boolean isValidName(String src) {
+     // Different local file systems have different validation rules.  Skip
+     // validation here and just let the OS handle it.  This is consistent with
+     // RawLocalFileSystem.
+     return true;
+   }
+  
   @Override
   @Override
   public Path getLinkTarget(Path f) throws IOException {
   public Path getLinkTarget(Path f) throws IOException {
     /* We should never get here. Valid local links are resolved transparently
     /* We should never get here. Valid local links are resolved transparently

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFs.java

@@ -83,7 +83,12 @@ class ChRootedFs extends AbstractFileSystem {
     return new Path((chRootPathPart.isRoot() ? "" : chRootPathPartString)
     return new Path((chRootPathPart.isRoot() ? "" : chRootPathPartString)
         + path.toUri().getPath());
         + path.toUri().getPath());
   }
   }
-  
+
+  @Override
+  public boolean isValidName(String src) {
+    return myFs.isValidName(fullPath(new Path(src)).toUri().toString());
+  }
+
   public ChRootedFs(final AbstractFileSystem fs, final Path theRoot)
   public ChRootedFs(final AbstractFileSystem fs, final Path theRoot)
     throws URISyntaxException {
     throws URISyntaxException {
     super(fs.getUri(), fs.getUri().getScheme(),
     super(fs.getUri(), fs.getUri().getScheme(),
@@ -103,7 +108,7 @@ class ChRootedFs extends AbstractFileSystem {
     //              scheme:/// and scheme://authority/
     //              scheme:/// and scheme://authority/
     myUri = new URI(myFs.getUri().toString() + 
     myUri = new URI(myFs.getUri().toString() + 
         (myFs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
         (myFs.getUri().getAuthority() == null ? "" :  Path.SEPARATOR) +
-          chRootPathPart.toString().substring(1));
+          chRootPathPart.toUri().getPath().substring(1));
     super.checkPath(theRoot);
     super.checkPath(theRoot);
   }
   }
   
   

+ 4 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -64,6 +64,9 @@ import org.apache.hadoop.util.Time;
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
 public class ViewFileSystem extends FileSystem {
 public class ViewFileSystem extends FileSystem {
+
+  private static final Path ROOT_PATH = new Path(Path.SEPARATOR);
+
   static AccessControlException readOnlyMountTable(final String operation,
   static AccessControlException readOnlyMountTable(final String operation,
       final String p) {
       final String p) {
     return new AccessControlException( 
     return new AccessControlException( 
@@ -98,23 +101,6 @@ public class ViewFileSystem extends FileSystem {
   InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
   InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
   Path homeDir = null;
   Path homeDir = null;
   
   
-  /**
-   * Prohibits names which contain a ".", "..", ":" or "/" 
-   */
-  private static boolean isValidName(final String src) {
-    // Check for ".." "." ":" "/"
-    final StringTokenizer tokens = new StringTokenizer(src, Path.SEPARATOR);
-    while(tokens.hasMoreTokens()) {
-      String element = tokens.nextToken();
-      if (element.equals("..") ||
-          element.equals(".")  ||
-          (element.indexOf(":") >= 0)) {
-        return false;
-      }
-    }
-    return true;
-  }
-  
   /**
   /**
    * Make the path Absolute and get the path-part of a pathname.
    * Make the path Absolute and get the path-part of a pathname.
    * Checks that URI matches this file system 
    * Checks that URI matches this file system 
@@ -126,10 +112,6 @@ public class ViewFileSystem extends FileSystem {
   private String getUriPath(final Path p) {
   private String getUriPath(final Path p) {
     checkPath(p);
     checkPath(p);
     String s = makeAbsolute(p).toUri().getPath();
     String s = makeAbsolute(p).toUri().getPath();
-    if (!isValidName(s)) {
-      throw new InvalidPathException("Path part " + s + " from URI" + p
-          + " is not a valid filename.");
-    }
     return s;
     return s;
   }
   }
   
   
@@ -689,7 +671,7 @@ public class ViewFileSystem extends FileSystem {
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
           PERMISSION_RRR, ugi.getUserName(), ugi.getGroupNames()[0],
 
 
           new Path(theInternalDir.fullPath).makeQualified(
           new Path(theInternalDir.fullPath).makeQualified(
-              myUri, null));
+              myUri, ROOT_PATH));
     }
     }
     
     
 
 

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java

@@ -597,6 +597,12 @@ public class ViewFs extends AbstractFileSystem {
     return result;
     return result;
   }
   }
 
 
+  @Override
+  public boolean isValidName(String src) {
+    // Prefix validated at mount time and rest of path validated by mount target.
+    return true;
+  }
+
   
   
   
   
   /*
   /*

+ 3 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Shell;
 
 
 import static org.apache.hadoop.fs.FileSystemTestHelper.*;
 import static org.apache.hadoop.fs.FileSystemTestHelper.*;
 
 
@@ -28,6 +29,7 @@ import java.util.Arrays;
 import java.util.Random;
 import java.util.Random;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
 
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -263,6 +265,7 @@ public class TestLocalFileSystem {
 
 
   @Test(timeout = 1000)
   @Test(timeout = 1000)
   public void testListStatusWithColons() throws IOException {
   public void testListStatusWithColons() throws IOException {
+    assumeTrue(!Shell.WINDOWS);
     File colonFile = new File(TEST_ROOT_DIR, "foo:bar");
     File colonFile = new File(TEST_ROOT_DIR, "foo:bar");
     colonFile.mkdirs();
     colonFile.mkdirs();
     FileStatus[] stats = fileSys.listStatus(new Path(TEST_ROOT_DIR));
     FileStatus[] stats = fileSys.listStatus(new Path(TEST_ROOT_DIR));

+ 19 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFs.java

@@ -25,6 +25,7 @@ import java.util.EnumSet;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextTestHelper;
 import org.apache.hadoop.fs.FileContextTestHelper;
@@ -36,6 +37,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 
 public class TestChRootedFs {
 public class TestChRootedFs {
   FileContextTestHelper fileContextTestHelper = new FileContextTestHelper();
   FileContextTestHelper fileContextTestHelper = new FileContextTestHelper();
@@ -308,4 +310,21 @@ public class TestChRootedFs {
       fc.getDefaultFileSystem().resolvePath(new Path("/nonExisting"));
       fc.getDefaultFileSystem().resolvePath(new Path("/nonExisting"));
   }
   }
  
  
+  @Test
+  public void testIsValidNameValidInBaseFs() throws Exception {
+    AbstractFileSystem baseFs = Mockito.spy(fc.getDefaultFileSystem());
+    ChRootedFs chRootedFs = new ChRootedFs(baseFs, new Path("/chroot"));
+    Mockito.doReturn(true).when(baseFs).isValidName(Mockito.anyString());
+    Assert.assertTrue(chRootedFs.isValidName("/test"));
+    Mockito.verify(baseFs).isValidName("/chroot/test");
+  }
+
+  @Test
+  public void testIsValidNameInvalidInBaseFs() throws Exception {
+    AbstractFileSystem baseFs = Mockito.spy(fc.getDefaultFileSystem());
+    ChRootedFs chRootedFs = new ChRootedFs(baseFs, new Path("/chroot"));
+    Mockito.doReturn(false).when(baseFs).isValidName(Mockito.anyString());
+    Assert.assertFalse(chRootedFs.isValidName("/test"));
+    Mockito.verify(baseFs).isValidName("/chroot/test");
+  }
 }
 }

+ 6 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.util.Shell;
 import org.mortbay.log.Log;
 import org.mortbay.log.Log;
 
 
 
 
@@ -130,8 +131,11 @@ public class ViewFileSystemTestSetup {
    * in the target file system.
    * in the target file system.
    */
    */
   static void linkUpFirstComponents(Configuration conf, String path, FileSystem fsTarget, String info) {
   static void linkUpFirstComponents(Configuration conf, String path, FileSystem fsTarget, String info) {
-    int indexOf2ndSlash = path.indexOf('/', 1);
-    String firstComponent = path.substring(0, indexOf2ndSlash);
+    int indexOfEnd = path.indexOf('/', 1);
+    if (Shell.WINDOWS) {
+      indexOfEnd = path.indexOf('/', indexOfEnd + 1);
+    }
+    String firstComponent = path.substring(0, indexOfEnd);
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
     Log.info("Added link for " + info + " " 
     Log.info("Added link for " + info + " " 

+ 6 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileContextTestHelper;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.util.Shell;
 import org.mortbay.log.Log;
 import org.mortbay.log.Log;
 
 
 
 
@@ -120,8 +121,11 @@ public class ViewFsTestSetup {
    */
    */
   static void linkUpFirstComponents(Configuration conf, String path,
   static void linkUpFirstComponents(Configuration conf, String path,
       FileContext fsTarget, String info) {
       FileContext fsTarget, String info) {
-    int indexOf2ndSlash = path.indexOf('/', 1);
-    String firstComponent = path.substring(0, indexOf2ndSlash);
+    int indexOfEnd = path.indexOf('/', 1);
+    if (Shell.WINDOWS) {
+      indexOfEnd = path.indexOf('/', indexOfEnd + 1);
+    }
+    String firstComponent = path.substring(0, indexOfEnd);
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
     Log.info("Added link for " + info + " " 
     Log.info("Added link for " + info + " " 

+ 21 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java

@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -110,9 +111,9 @@ public class TestShellCommandFencer {
    */
    */
   @Test
   @Test
   public void testStderrLogging() {
   public void testStderrLogging() {
-    assertTrue(fencer.tryFence(TEST_TARGET, "echo hello >&2"));
+    assertTrue(fencer.tryFence(TEST_TARGET, "echo hello>&2"));
     Mockito.verify(ShellCommandFencer.LOG).warn(
     Mockito.verify(ShellCommandFencer.LOG).warn(
-        Mockito.endsWith("echo hello >&2: hello"));
+        Mockito.endsWith("echo hello>&2: hello"));
   }
   }
 
 
   /**
   /**
@@ -121,9 +122,15 @@ public class TestShellCommandFencer {
    */
    */
   @Test
   @Test
   public void testConfAsEnvironment() {
   public void testConfAsEnvironment() {
-    fencer.tryFence(TEST_TARGET, "echo $in_fencing_tests");
-    Mockito.verify(ShellCommandFencer.LOG).info(
-        Mockito.endsWith("echo $in...ing_tests: yessir"));
+    if (!Shell.WINDOWS) {
+      fencer.tryFence(TEST_TARGET, "echo $in_fencing_tests");
+      Mockito.verify(ShellCommandFencer.LOG).info(
+          Mockito.endsWith("echo $in...ing_tests: yessir"));
+    } else {
+      fencer.tryFence(TEST_TARGET, "echo %in_fencing_tests%");
+      Mockito.verify(ShellCommandFencer.LOG).info(
+          Mockito.endsWith("echo %in...ng_tests%: yessir"));
+    }
   }
   }
   
   
   /**
   /**
@@ -132,9 +139,15 @@ public class TestShellCommandFencer {
    */
    */
   @Test
   @Test
   public void testTargetAsEnvironment() {
   public void testTargetAsEnvironment() {
-    fencer.tryFence(TEST_TARGET, "echo $target_host $target_port $target_address");
-    Mockito.verify(ShellCommandFencer.LOG).info(
-        Mockito.endsWith("echo $ta...t_address: host 1234 host:1234"));
+    if (!Shell.WINDOWS) {
+      fencer.tryFence(TEST_TARGET, "echo $target_host $target_port $target_address");
+      Mockito.verify(ShellCommandFencer.LOG).info(
+          Mockito.endsWith("echo $ta...t_address: host 1234 host:1234"));
+    } else {
+      fencer.tryFence(TEST_TARGET, "echo %target_host% %target_port% %target_address%");
+      Mockito.verify(ShellCommandFencer.LOG).info(
+          Mockito.endsWith("echo %ta..._address%: host 1234 host:1234"));
+    }
   }
   }
 
 
 
 

+ 15 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.test.MultithreadedTestUtil;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Shell;
 
 
 import org.junit.Test;
 import org.junit.Test;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
@@ -144,12 +145,20 @@ public class TestSocketIOWithTimeout {
       // Nevertheless, the output stream is closed, because
       // Nevertheless, the output stream is closed, because
       // a partial write may have succeeded (see comment in
       // a partial write may have succeeded (see comment in
       // SocketOutputStream#write(byte[]), int, int)
       // SocketOutputStream#write(byte[]), int, int)
-      try {
-        out.write(1);
-        fail("Did not throw");
-      } catch (IOException ioe) {
-        GenericTestUtils.assertExceptionContains(
-            "stream is closed", ioe);
+      // This portion of the test cannot pass on Windows due to differences in
+      // behavior of partial writes.  Windows appears to buffer large amounts of
+      // written data and send it all atomically, thus making it impossible to
+      // simulate a partial write scenario.  Attempts were made to switch the
+      // test from using a pipe to a network socket and also to use larger and
+      // larger buffers in doIO.  Nothing helped the situation though.
+      if (!Shell.WINDOWS) {
+        try {
+          out.write(1);
+          fail("Did not throw");
+        } catch (IOException ioe) {
+          GenericTestUtils.assertExceptionContains(
+              "stream is closed", ioe);
+        }
       }
       }
       
       
       out.close();
       out.close();

+ 6 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java

@@ -41,7 +41,12 @@ public class TestShell extends TestCase {
 
 
     @Override
     @Override
     protected String[] getExecString() {
     protected String[] getExecString() {
-      return new String[] {"echo", "hello"};
+      // There is no /bin/echo equivalent on Windows so just launch it as a
+      // shell built-in.
+      //
+      return Shell.WINDOWS ?
+          (new String[] {"cmd.exe", "/c", "echo", "hello"}) :
+          (new String[] {"echo", "hello"});
     }
     }
 
 
     @Override
     @Override

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

@@ -318,6 +318,9 @@ Release 2.1.0-beta - UNRELEASED
 
 
     HDFS-4840. ReplicationMonitor gets NPE during shutdown. (kihwal)
     HDFS-4840. ReplicationMonitor gets NPE during shutdown. (kihwal)
 
 
+    HADOOP-8957 HDFS tests for AbstractFileSystem#IsValidName should be overridden for
+    embedded file systems like ViewFs (Chris Nauroth via Sanjay Radia)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

+ 16 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java

@@ -257,7 +257,22 @@ public class TestHDFSFileContextMainOperations extends
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
     Assert.assertTrue(fs.exists(dst1));    // ensure rename dst exists
     Assert.assertTrue(fs.exists(dst1));    // ensure rename dst exists
   }
   }
-  
+
+  @Test
+  public void testIsValidNameInvalidNames() {
+    String[] invalidNames = {
+      "/foo/../bar",
+      "/foo/./bar",
+      "/foo/:/bar",
+      "/foo:bar"
+    };
+
+    for (String invalidName: invalidNames) {
+      Assert.assertFalse(invalidName + " is not valid",
+        fc.getDefaultFileSystem().isValidName(invalidName));
+    }
+  }
+
   private void oldRename(Path src, Path dst, boolean renameSucceeds,
   private void oldRename(Path src, Path dst, boolean renameSucceeds,
       boolean exception) throws Exception {
       boolean exception) throws Exception {
     DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
     DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -640,9 +640,12 @@ public class TestDFSUtil {
   @Test (timeout=15000)
   @Test (timeout=15000)
   public void testIsValidName() {
   public void testIsValidName() {
     assertFalse(DFSUtil.isValidName("/foo/../bar"));
     assertFalse(DFSUtil.isValidName("/foo/../bar"));
+    assertFalse(DFSUtil.isValidName("/foo/./bar"));
     assertFalse(DFSUtil.isValidName("/foo//bar"));
     assertFalse(DFSUtil.isValidName("/foo//bar"));
     assertTrue(DFSUtil.isValidName("/"));
     assertTrue(DFSUtil.isValidName("/"));
     assertTrue(DFSUtil.isValidName("/bar/"));
     assertTrue(DFSUtil.isValidName("/bar/"));
+    assertFalse(DFSUtil.isValidName("/foo/:/bar"));
+    assertFalse(DFSUtil.isValidName("/foo:bar"));
   }
   }
   
   
   @Test(timeout=5000)
   @Test(timeout=5000)