浏览代码

Merging r1520450 through r1521565 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1521566 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 12 年之前
父节点
当前提交
23bf148a2f
共有 100 个文件被更改,包括 1457 次插入372 次删除
  1. 2 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 3
      hadoop-common-project/hadoop-common/pom.xml
  3. 25 0
      hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
  4. 3 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  5. 8 48
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
  6. 94 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java
  7. 62 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java
  8. 15 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
  9. 62 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java
  10. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
  11. 0 15
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
  12. 4 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
  13. 0 20
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java
  14. 5 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
  15. 47 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java
  16. 37 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java
  17. 22 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java
  18. 14 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  19. 39 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  20. 26 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  21. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  22. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  24. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  26. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  28. 8 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  29. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  30. 171 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java
  31. 14 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
  32. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  33. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/fs/TestFiRename.java
  34. 0 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
  35. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java
  36. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java
  37. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
  38. 22 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  39. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
  40. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  41. 5 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  42. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  43. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  44. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
  45. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
  46. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
  47. 7 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
  48. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
  49. 5 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
  50. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
  51. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
  52. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  53. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
  54. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  55. 32 27
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
  56. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  57. 13 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java
  58. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
  59. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
  60. 9 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  61. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
  62. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  63. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java
  64. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
  65. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  66. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  67. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  68. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
  69. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java
  70. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  71. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
  72. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
  73. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
  74. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java
  75. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
  76. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
  77. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java
  78. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
  79. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
  80. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java
  81. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
  82. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestAtomicFileOutputStream.java
  83. 93 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java
  84. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestMD5FileUtils.java
  85. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java
  86. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java
  87. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  88. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
  89. 2 20
      hadoop-project/pom.xml
  90. 29 3
      hadoop-yarn-project/CHANGES.txt
  91. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java
  92. 24 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/AuxiliaryService.java
  93. 75 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
  94. 44 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
  95. 44 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
  96. 17 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
  97. 19 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEvent.java
  98. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEventType.java
  99. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  100. 59 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java

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

@@ -357,8 +357,6 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9865.  FileContext#globStatus has a regression with respect to
     relative path.  (Chuan Lin via Colin Patrick McCabe)
 
-    HADOOP-9877. Fix listing of snapshot directories in globStatus.
-    (Binglin Chang via Andrew Wang)
 
     HADOOP-9909. org.apache.hadoop.fs.Stat should permit other LANG.
     (Shinichi Yamashita via Andrew Wang)
@@ -2117,6 +2115,8 @@ Release 0.23.10 - UNRELEASED
     HADOOP-9686. Easy access to final parameters in Configuration (Jason Lowe
     via jeagles)
 
+    HADOOP-8704. add request logging to jetty/httpserver (jeagles)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -780,9 +780,7 @@
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-surefire-plugin</artifactId>
             <configuration>
-              <forkMode>perthread</forkMode>
-              <threadCount>${testsThreadCount}</threadCount>
-              <parallel>classes</parallel>
+              <forkCount>${testsThreadCount}</forkCount>
               <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -DminiClusterDedicatedDirs=true</argLine>
             </configuration>
           </plugin>

+ 25 - 0
hadoop-common-project/hadoop-common/src/main/conf/log4j.properties

@@ -240,3 +240,28 @@ log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
 #log4j.appender.HSAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
 #log4j.appender.HSAUDIT.DatePattern=.yyyy-MM-dd
 
+# Http Server Request Logs
+#log4j.logger.http.requests.namenode=INFO,namenoderequestlog
+#log4j.appender.namenoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.namenoderequestlog.Filename=${hadoop.log.dir}/jetty-namenode-yyyy_mm_dd.log
+#log4j.appender.namenoderequestlog.RetainDays=3
+
+#log4j.logger.http.requests.datanode=INFO,datanoderequestlog
+#log4j.appender.datanoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.datanoderequestlog.Filename=${hadoop.log.dir}/jetty-datanode-yyyy_mm_dd.log
+#log4j.appender.datanoderequestlog.RetainDays=3
+
+#log4j.logger.http.requests.resourcemanager=INFO,resourcemanagerrequestlog
+#log4j.appender.resourcemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.resourcemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-resourcemanager-yyyy_mm_dd.log
+#log4j.appender.resourcemanagerrequestlog.RetainDays=3
+
+#log4j.logger.http.requests.jobhistory=INFO,jobhistoryrequestlog
+#log4j.appender.jobhistoryrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.jobhistoryrequestlog.Filename=${hadoop.log.dir}/jetty-jobhistory-yyyy_mm_dd.log
+#log4j.appender.jobhistoryrequestlog.RetainDays=3
+
+#log4j.logger.http.requests.nodemanager=INFO,nodemanagerrequestlog
+#log4j.appender.nodemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.nodemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-nodemanager-yyyy_mm_dd.log
+#log4j.appender.nodemanagerrequestlog.RetainDays=3

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -1239,6 +1239,9 @@ public class FileUtil {
     List<String> classPathEntryList = new ArrayList<String>(
       classPathEntries.length);
     for (String classPathEntry: classPathEntries) {
+      if (classPathEntry.length() == 0) {
+        continue;
+      }
       if (classPathEntry.endsWith("*")) {
         // Append all jars that match the wildcard
         Path globPath = new Path(classPathEntry).suffix("{.jar,.JAR}");

+ 8 - 48
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java

@@ -62,18 +62,6 @@ class Globber {
     }
   }
 
-  private FileStatus getFileLinkStatus(Path path) {
-    try {
-      if (fs != null) {
-        return fs.getFileLinkStatus(path);
-      } else {
-        return fc.getFileLinkStatus(path);
-      }
-    } catch (IOException e) {
-      return null;
-    }
-  }
-
   private FileStatus[] listStatus(Path path) {
     try {
       if (fs != null) {
@@ -134,18 +122,6 @@ class Globber {
     return authority ;
   }
 
-  /**
-   * The glob filter builds a regexp per path component.  If the component
-   * does not contain a shell metachar, then it falls back to appending the
-   * raw string to the list of built up paths.  This raw path needs to have
-   * the quoting removed.  Ie. convert all occurrences of "\X" to "X"
-   * @param name of the path component
-   * @return the unquoted path component
-   */
-  private static String unquotePathComponent(String name) {
-    return name.replaceAll("\\\\(.)", "$1");
-  }
-
   public FileStatus[] glob() throws IOException {
     // First we get the scheme and authority of the pattern that was passed
     // in.
@@ -210,30 +186,14 @@ class Globber {
               resolvedCandidate.isDirectory() == false) {
             continue;
           }
-          // For components without pattern, we get its FileStatus directly
-          // using getFileLinkStatus for two reasons:
-          // 1. It should be faster to only get FileStatus needed rather than
-          //    get all children.
-          // 2. Some special filesystem directories (e.g. HDFS snapshot
-          //    directories) are not returned by listStatus, but do exist if
-          //    checked explicitly via getFileLinkStatus.
-          if (globFilter.hasPattern()) {
-            FileStatus[] children = listStatus(candidate.getPath());
-            for (FileStatus child : children) {
-              // Set the child path based on the parent path.
-              // This keeps the symlinks in our path.
-              child.setPath(new Path(candidate.getPath(),
-                      child.getPath().getName()));
-              if (globFilter.accept(child.getPath())) {
-                newCandidates.add(child);
-              }
-            }
-          } else {
-            Path p = new Path(candidate.getPath(), unquotePathComponent(component));
-            FileStatus s = getFileLinkStatus(p);
-            if (s != null) {
-              s.setPath(p);
-              newCandidates.add(s);
+          FileStatus[] children = listStatus(candidate.getPath());
+          for (FileStatus child : children) {
+            // Set the child path based on the parent path.
+            // This keeps the symlinks in our path.
+            child.setPath(new Path(candidate.getPath(),
+                    child.getPath().getName()));
+            if (globFilter.accept(child.getPath())) {
+              newCandidates.add(child);
             }
           }
         }

+ 94 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java

@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+import java.util.HashMap;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogConfigurationException;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Appender;
+import org.apache.log4j.Logger;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
+
+/**
+ * RequestLog object for use with Http
+ */
+public class HttpRequestLog {
+
+  public static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
+  private static final HashMap<String, String> serverToComponent;
+
+  static {
+    serverToComponent = new HashMap<String, String>();
+    serverToComponent.put("cluster", "resourcemanager");
+    serverToComponent.put("hdfs", "namenode");
+    serverToComponent.put("node", "nodemanager");
+  }
+
+  public static RequestLog getRequestLog(String name) {
+
+    String lookup = serverToComponent.get(name);
+    if (lookup != null) {
+      name = lookup;
+    }
+    String loggerName = "http.requests." + name;
+    String appenderName = name + "requestlog";
+    Log logger = LogFactory.getLog(loggerName);
+
+    if (logger instanceof Log4JLogger) {
+      Log4JLogger httpLog4JLog = (Log4JLogger)logger;
+      Logger httpLogger = httpLog4JLog.getLogger();
+      Appender appender = null;
+
+      try {
+        appender = httpLogger.getAppender(appenderName);
+      } catch (LogConfigurationException e) {
+        LOG.warn("Http request log for " + loggerName
+            + " could not be created");
+        throw e;
+      }
+
+      if (appender == null) {
+        LOG.info("Http request log for " + loggerName
+            + " is not defined");
+        return null;
+      }
+
+      if (appender instanceof HttpRequestLogAppender) {
+        HttpRequestLogAppender requestLogAppender
+          = (HttpRequestLogAppender)appender;
+        NCSARequestLog requestLog = new NCSARequestLog();
+        requestLog.setFilename(requestLogAppender.getFilename());
+        requestLog.setRetainDays(requestLogAppender.getRetainDays());
+        return requestLog;
+      }
+      else {
+        LOG.warn("Jetty request log for " + loggerName
+            + " was of the wrong class");
+        return null;
+      }
+    }
+    else {
+      LOG.warn("Jetty request log can only be enabled using Log4j");
+      return null;
+    }
+  }
+}

+ 62 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+import org.apache.log4j.spi.LoggingEvent;
+import org.apache.log4j.AppenderSkeleton;
+
+/**
+ * Log4j Appender adapter for HttpRequestLog
+ */
+public class HttpRequestLogAppender extends AppenderSkeleton {
+
+  private String filename;
+  private int retainDays;
+
+  public HttpRequestLogAppender() {
+  }
+
+  public void setRetainDays(int retainDays) {
+    this.retainDays = retainDays;
+  }
+
+  public int getRetainDays() {
+    return retainDays;
+  }
+
+  public void setFilename(String filename) {
+    this.filename = filename;
+  }
+
+  public String getFilename() {
+    return filename;
+  }
+
+  @Override
+  public void append(LoggingEvent event) {
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public boolean requiresLayout() {
+    return false;
+  }
+}

+ 15 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java

@@ -67,9 +67,12 @@ import org.mortbay.io.Buffer;
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.MimeTypes;
+import org.mortbay.jetty.RequestLog;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.handler.ContextHandler;
 import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.handler.RequestLogHandler;
+import org.mortbay.jetty.handler.HandlerCollection;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.security.SslSocketConnector;
 import org.mortbay.jetty.servlet.Context;
@@ -355,7 +358,18 @@ public class HttpServer implements FilterContainer {
 
     final String appDir = getWebAppsPath(name);
     ContextHandlerCollection contexts = new ContextHandlerCollection();
-    webServer.setHandler(contexts);
+    RequestLog requestLog = HttpRequestLog.getRequestLog(name);
+
+    if (requestLog != null) {
+      RequestLogHandler requestLogHandler = new RequestLogHandler();
+      requestLogHandler.setRequestLog(requestLog);
+      HandlerCollection handlers = new HandlerCollection();
+      handlers.setHandlers(new Handler[] {requestLogHandler, contexts});
+      webServer.setHandler(handlers);
+    }
+    else {
+      webServer.setHandler(contexts);
+    }
 
     webAppContext = new WebAppContext();
     webAppContext.setDisplayName(name);

+ 62 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.retry;
+
+import java.lang.reflect.Method;
+import java.net.UnknownHostException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A dummy invocation handler extending RetryInvocationHandler. It drops the
+ * first N number of responses. This invocation handler is only used for testing.
+ */
+@InterfaceAudience.Private
+public class LossyRetryInvocationHandler<T> extends RetryInvocationHandler<T> {
+  private final int numToDrop;
+  private static final ThreadLocal<Integer> RetryCount = 
+      new ThreadLocal<Integer>();
+
+  public LossyRetryInvocationHandler(int numToDrop,
+      FailoverProxyProvider<T> proxyProvider, RetryPolicy retryPolicy) {
+    super(proxyProvider, retryPolicy);
+    this.numToDrop = numToDrop;
+  }
+
+  @Override
+  public Object invoke(Object proxy, Method method, Object[] args)
+      throws Throwable {
+    RetryCount.set(0);
+    return super.invoke(proxy, method, args);
+  }
+
+  @Override
+  protected Object invokeMethod(Method method, Object[] args) throws Throwable {
+    Object result = super.invokeMethod(method, args);
+    int retryCount = RetryCount.get();
+    if (retryCount < this.numToDrop) {
+      RetryCount.set(++retryCount);
+      LOG.info("Drop the response. Current retryCount == " + retryCount);
+      throw new UnknownHostException("Fake Exception");
+    } else {
+      LOG.info("retryCount == " + retryCount
+          + ". It's time to normally process the response");
+      return result;
+    }
+  }
+}

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -63,7 +63,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
     this(proxyProvider, retryPolicy, Collections.<String, RetryPolicy>emptyMap());
   }
 
-  RetryInvocationHandler(FailoverProxyProvider<T> proxyProvider,
+  protected RetryInvocationHandler(FailoverProxyProvider<T> proxyProvider,
       RetryPolicy defaultPolicy,
       Map<String, RetryPolicy> methodNameToPolicyMap) {
     this.proxyProvider = proxyProvider;

+ 0 - 15
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.After;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -633,20 +632,6 @@ public abstract class FileContextMainOperationsBaseTest  {
         filteredPaths));
   }
   
-  protected Path getHiddenPathForTest() {
-    return null;
-  }
-  
-  @Test
-  public void testGlobStatusFilterWithHiddenPathTrivialFilter()
-      throws Exception {
-    Path hidden = getHiddenPathForTest();
-    Assume.assumeNotNull(hidden);
-    FileStatus[] filteredPaths = fc.util().globStatus(hidden, DEFAULT_FILTER);
-    Assert.assertNotNull(filteredPaths);
-    Assert.assertEquals(1, filteredPaths.length);
-  }
-
   @Test
   public void testWriteReadAndDeleteEmptyFile() throws Exception {
     writeReadAndDelete(0);

+ 4 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java

@@ -757,7 +757,7 @@ public class TestFileUtil {
     String wildcardPath = tmp.getCanonicalPath() + File.separator + "*";
     String nonExistentSubdir = tmp.getCanonicalPath() + Path.SEPARATOR + "subdir"
       + Path.SEPARATOR;
-    List<String> classPaths = Arrays.asList("cp1.jar", "cp2.jar", wildcardPath,
+    List<String> classPaths = Arrays.asList("", "cp1.jar", "cp2.jar", wildcardPath,
       "cp3.jar", nonExistentSubdir);
     String inputClassPath = StringUtils.join(File.pathSeparator, classPaths);
     String classPathJar = FileUtil.createJarWithClassPath(inputClassPath,
@@ -776,6 +776,9 @@ public class TestFileUtil {
       Assert.assertNotNull(classPathAttr);
       List<String> expectedClassPaths = new ArrayList<String>();
       for (String classPath: classPaths) {
+        if (classPath.length() == 0) {
+          continue;
+        }
         if (wildcardPath.equals(classPath)) {
           // add wildcard matches
           for (File wildcardMatch: wildcardMatches) {

+ 0 - 20
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellReturnCode.java

@@ -517,26 +517,6 @@ public class TestFsShellReturnCode {
       }
       return stat;
     }
-
-    @Override
-    public FileStatus getFileLinkStatus(Path p) throws IOException {
-      String f = makeQualified(p).toString();
-      FileStatus stat = super.getFileLinkStatus(p);
-      
-      stat.getPermission();
-      if (owners.containsKey(f)) {
-        stat.setOwner("STUB-"+owners.get(f));      
-      } else {
-        stat.setOwner("REAL-"+stat.getOwner());
-      }
-      if (groups.containsKey(f)) {
-        stat.setGroup("STUB-"+groups.get(f));      
-      } else {
-        stat.setGroup("REAL-"+stat.getGroup());
-      }
-      return stat;
-    }
-
   }
   
   static class MyFsShell extends FsShell {

+ 5 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java

@@ -101,8 +101,12 @@ public class HttpServerFunctionalTest extends Assert {
     String webapps = System.getProperty(TEST_BUILD_WEBAPPS, BUILD_WEBAPPS_DIR);
     File testWebappDir = new File(webapps +
         File.separatorChar + TEST);
+    try {
     if (!testWebappDir.exists()) {
-      fail("Test webapp dir " + testWebappDir + " missing");
+      fail("Test webapp dir " + testWebappDir.getCanonicalPath() + " missing");
+    }
+    }
+    catch (IOException e) {
     }
   }
 

+ 47 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+import org.apache.log4j.Logger;
+import org.junit.Test;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestHttpRequestLog {
+
+  @Test
+  public void testAppenderUndefined() {
+    RequestLog requestLog = HttpRequestLog.getRequestLog("test");
+    assertNull("RequestLog should be null", requestLog);
+  }
+
+  @Test
+  public void testAppenderDefined() {
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setName("testrequestlog");
+    Logger.getLogger("http.requests.test").addAppender(requestLogAppender);
+    RequestLog requestLog = HttpRequestLog.getRequestLog("test");
+    Logger.getLogger("http.requests.test").removeAppender(requestLogAppender);
+    assertNotNull("RequestLog should not be null", requestLog);
+    assertEquals("Class mismatch", NCSARequestLog.class, requestLog.getClass());
+  }
+}

+ 37 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.http;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHttpRequestLogAppender {
+
+  @Test
+  public void testParameterPropagation() {
+
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setFilename("jetty-namenode-yyyy_mm_dd.log");
+    requestLogAppender.setRetainDays(17);
+    assertEquals("Filename mismatch", "jetty-namenode-yyyy_mm_dd.log",
+        requestLogAppender.getFilename());
+    assertEquals("Retain days mismatch", 17,
+        requestLogAppender.getRetainDays());
+  }
+}

+ 22 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.http;
 
+import org.apache.log4j.Logger;
 import org.junit.Test;
 
 public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
@@ -66,6 +67,27 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
     stop(server);
   }
 
+  /**
+   * Test that the server with request logging enabled
+   *
+   * @throws Throwable on failure
+   */
+  @Test
+  public void testStartedServerWithRequestLog() throws Throwable {
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setName("httprequestlog");
+    requestLogAppender.setFilename(System.getProperty("test.build.data", "/tmp/")
+        + "jetty-name-yyyy_mm_dd.log");
+    Logger.getLogger(HttpServer.class.getName() + ".test").addAppender(requestLogAppender);
+    HttpServer server = null;
+    server = createTestServer();
+    assertNotLive(server);
+    server.start();
+    assertAlive(server);
+    stop(server);
+    Logger.getLogger(HttpServer.class.getName() + ".test").removeAppender(requestLogAppender);
+  }
+
   /**
    * Assert that the result of {@link HttpServer#toString()} contains the specific text
    * @param server server to examine

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

@@ -267,6 +267,11 @@ Release 2.3.0 - UNRELEASED
     HDFS-5144. Document time unit to NameNodeMetrics. (Akira Ajisaka via
     suresh)
 
+    HDFS-4491. Parallel testing HDFS. (Andrey Klochkov via cnauroth)
+
+    HDFS-4879. Add "blocked ArrayList" collection to avoid CMS full GCs
+    (Todd Lipcon via Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -281,6 +286,12 @@ Release 2.3.0 - UNRELEASED
     HDFS-4816. transitionToActive blocks if the SBN is doing checkpoint image
     transfer. (Andrew Wang)
 
+    HDFS-5164.  deleteSnapshot should check if OperationCategory.WRITE is
+    possible before taking write lock.  (Colin Patrick McCabe)
+
+    HDFS-5170. BlockPlacementPolicyDefault uses the wrong classname when
+    alerting to enable debug logging. (Andrew Wang)
+
 Release 2.1.1-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -316,6 +327,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5136 MNT EXPORT should give the full group list which can mount the
     exports (brandonli)
 
+    HDFS-5118. Provide testing support for DFSClient to drop RPC responses.
+    (jing9)
+
   IMPROVEMENTS
 
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -700,5 +700,44 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>parallel-tests</id>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-parallel-tests-dirs</id>               
+                <phase>test-compile</phase>
+                <configuration>
+                  <target>
+                    <exec executable="sh">
+                      <arg value="-c"/>
+                      <arg value="for i in {1..${testsThreadCount}}; do mkdir -p ${test.build.data}/$i; mkdir -p ${hadoop.tmp.dir}/$i; done"/>
+                    </exec>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <configuration>
+              <forkCount>${testsThreadCount}</forkCount>
+              <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -DminiClusterDedicatedDirs=true</argLine>
+              <systemPropertyVariables>
+                <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+              </systemPropertyVariables>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 </project>

+ 26 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -27,6 +27,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
@@ -44,9 +47,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIR
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
@@ -100,6 +100,7 @@ import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -113,13 +114,13 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -144,6 +145,7 @@ import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -451,7 +453,11 @@ public class DFSClient implements java.io.Closeable {
   
   /** 
    * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
-   * Exactly one of nameNodeUri or rpcNamenode must be null.
+   * If HA is enabled and a positive value is set for 
+   * {@link DFSConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY} in the
+   * configuration, the DFSClient will use {@link LossyRetryInvocationHandler}
+   * as its RetryInvocationHandler. Otherwise one of nameNodeUri or rpcNamenode 
+   * must be null.
    */
   @VisibleForTesting
   public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
@@ -475,7 +481,20 @@ public class DFSClient implements java.io.Closeable {
     this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" + 
         DFSUtil.getRandom().nextInt()  + "_" + Thread.currentThread().getId();
     
-    if (rpcNamenode != null) {
+    int numResponseToDrop = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
+        DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
+    if (numResponseToDrop > 0) {
+      // This case is used for testing.
+      LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
+          + " is set to " + numResponseToDrop
+          + ", this hacked client will proactively drop responses");
+      NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = NameNodeProxies
+          .createProxyWithLossyRetryHandler(conf, nameNodeUri,
+              ClientProtocol.class, numResponseToDrop);
+      this.dtService = proxyInfo.getDelegationTokenService();
+      this.namenode = proxyInfo.getProxy();
+    } else if (rpcNamenode != null) {
       // This case is used for testing.
       Preconditions.checkArgument(nameNodeUri == null);
       this.namenode = rpcNamenode;
@@ -514,7 +533,7 @@ public class DFSClient implements java.io.Closeable {
     this.defaultWriteCachingStrategy =
         new CachingStrategy(writeDropBehind, readahead);
   }
-
+  
   /**
    * Return the socket addresses to use with each configured
    * local interface. Local interfaces may be specified by IP

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -497,6 +497,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 600000; // 10 minutes
   public static final String DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY = "dfs.namenode.retrycache.heap.percent";
   public static final float DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT = 0.03f;
+  
+  // The number of NN response dropped by client proactively in each RPC call.
+  // For testing NN retry cache, we can set this property with positive value.
+  public static final String DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY = "dfs.client.test.drop.namenode.response.number";
+  public static final int DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT = 0;
+
 
   // Hidden configuration undocumented in hdfs-site. xml
   // Timeout to wait for block receiver and responder thread to stop

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
-import org.apache.hadoop.hdfs.web.URLUtils;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -86,6 +86,8 @@ public class HftpFileSystem extends FileSystem
     HttpURLConnection.setFollowRedirects(true);
   }
 
+  URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
+
   public static final Text TOKEN_KIND = new Text("HFTP delegation");
 
   protected UserGroupInformation ugi;
@@ -331,8 +333,8 @@ public class HftpFileSystem extends FileSystem
       throws IOException {
     query = addDelegationTokenParam(query);
     final URL url = getNamenodeURL(path, query);
-    final HttpURLConnection connection =
-        (HttpURLConnection)URLUtils.openConnection(url);
+    final HttpURLConnection connection;
+    connection = (HttpURLConnection)connectionFactory.openConnection(url);
     connection.setRequestMethod("GET");
     connection.connect();
     return connection;
@@ -352,12 +354,14 @@ public class HftpFileSystem extends FileSystem
   }
 
   static class RangeHeaderUrlOpener extends ByteRangeInputStream.URLOpener {
+    URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
+    
     RangeHeaderUrlOpener(final URL url) {
       super(url);
     }
 
     protected HttpURLConnection openConnection() throws IOException {
-      return (HttpURLConnection)URLUtils.openConnection(url);
+      return (HttpURLConnection)connectionFactory.openConnection(url);
     }
 
     /** Use HTTP Range header for specifying offset. */

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java

@@ -40,7 +40,6 @@ import javax.net.ssl.X509TrustManager;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.web.URLUtils;
 import org.apache.hadoop.util.Time;
 
 /**
@@ -154,7 +153,8 @@ public class HsftpFileSystem extends HftpFileSystem {
     query = addDelegationTokenParam(query);
     final URL url = new URL("https", nnUri.getHost(), 
         nnUri.getPort(), path + '?' + query);
-    HttpsURLConnection conn = (HttpsURLConnection)URLUtils.openConnection(url);
+    HttpsURLConnection conn;
+    conn = (HttpsURLConnection)connectionFactory.openConnection(url);
     // bypass hostname verification
     conn.setHostnameVerifier(new DummyHostnameVerifier());
     conn.setRequestMethod("GET");

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -17,10 +17,18 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Proxy;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.HashMap;
@@ -48,6 +56,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -144,6 +153,61 @@ public class NameNodeProxies {
       return new ProxyAndInfo<T>(proxy, dtService);
     }
   }
+  
+  /**
+   * Generate a dummy namenode proxy instance that utilizes our hacked
+   * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
+   * method will proactively drop RPC responses. Currently this method only
+   * support HA setup. IllegalStateException will be thrown if the given
+   * configuration is not for HA.
+   * 
+   * @param config the configuration containing the required IPC
+   *        properties, client failover configurations, etc.
+   * @param nameNodeUri the URI pointing either to a specific NameNode
+   *        or to a logical nameservice.
+   * @param xface the IPC interface which should be created
+   * @param numResponseToDrop The number of responses to drop for each RPC call
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to
+   * @throws IOException if there is an error creating the proxy
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler(
+      Configuration config, URI nameNodeUri, Class<T> xface,
+      int numResponseToDrop) throws IOException {
+    Preconditions.checkArgument(numResponseToDrop > 0);
+    Class<FailoverProxyProvider<T>> failoverProxyProviderClass = 
+        getFailoverProxyProviderClass(config, nameNodeUri, xface);
+    if (failoverProxyProviderClass != null) { // HA case
+      FailoverProxyProvider<T> failoverProxyProvider = 
+          createFailoverProxyProvider(config, failoverProxyProviderClass, 
+              xface, nameNodeUri);
+      int delay = config.getInt(
+          DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
+          DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
+      int maxCap = config.getInt(
+          DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
+          DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
+      int maxFailoverAttempts = config.getInt(
+          DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
+          DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
+      InvocationHandler dummyHandler = new LossyRetryInvocationHandler<T>(
+              numResponseToDrop, failoverProxyProvider,
+              RetryPolicies.failoverOnNetworkException(
+                  RetryPolicies.TRY_ONCE_THEN_FAIL, 
+                  Math.max(numResponseToDrop + 1, maxFailoverAttempts), delay, 
+                  maxCap));
+      
+      T proxy = (T) Proxy.newProxyInstance(
+          failoverProxyProvider.getInterface().getClassLoader(),
+          new Class[] { xface }, dummyHandler);
+      Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri);
+      return new ProxyAndInfo<T>(proxy, dtService);
+    } else {
+      throw new IllegalStateException("Currently creating proxy using " +
+      		"LossyRetryInvocationHandler requires NN HA setup");
+    }
+  }
 
   /**
    * Creates an explicitly non-HA-enabled proxy object. Most of the time you

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -55,7 +55,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
   private static final String enableDebugLogging =
     "For more information, please enable DEBUG log level on "
-    + LOG.getClass().getName();
+    + BlockPlacementPolicy.class.getName();
 
   protected boolean considerLoad; 
   private boolean preferLocalNode = true;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -70,6 +70,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapsho
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.util.ByteArray;
+import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -950,7 +951,7 @@ public class FSDirectory implements Closeable {
         if (removedDst != null) {
           undoRemoveDst = false;
           BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-          List<INode> removedINodes = new ArrayList<INode>();
+          List<INode> removedINodes = new ChunkedArrayList<INode>();
           filesDeleted = removedDst.cleanSubtree(null,
               dstIIP.getLatestSnapshot(), collectedBlocks, removedINodes, true)
               .get(Quota.NAMESPACE);
@@ -1364,7 +1365,7 @@ public class FSDirectory implements Closeable {
       QuotaExceededException, SnapshotAccessControlException {
     assert hasWriteLock();
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-    List<INode> removedINodes = new ArrayList<INode>();
+    List<INode> removedINodes = new ChunkedArrayList<INode>();
 
     final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
         normalizePath(src), false);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -22,7 +22,6 @@ import static org.apache.hadoop.util.Time.now;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.List;
@@ -75,6 +74,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
+import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.Holder;
 
 import com.google.common.base.Joiner;
@@ -582,7 +582,7 @@ public class FSEditLogLoader {
     case OP_DELETE_SNAPSHOT: {
       DeleteSnapshotOp deleteSnapshotOp = (DeleteSnapshotOp) op;
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-      List<INode> removedINodes = new ArrayList<INode>();
+      List<INode> removedINodes = new ChunkedArrayList<INode>();
       fsNamesys.getSnapshotManager().deleteSnapshot(
           deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
           collectedBlocks, removedINodes);

+ 8 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -208,6 +208,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RetryCache;
@@ -3143,7 +3144,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, SafeModeException, UnresolvedLinkException,
              IOException {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-    List<INode> removedINodes = new ArrayList<INode>();
+    List<INode> removedINodes = new ChunkedArrayList<INode>();
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -3197,21 +3198,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *          of blocks that need to be removed from blocksMap
    */
   void removeBlocks(BlocksMapUpdateInfo blocks) {
-    int start = 0;
-    int end = 0;
     List<Block> toDeleteList = blocks.getToDeleteList();
-    while (start < toDeleteList.size()) {
-      end = BLOCK_DELETION_INCREMENT + start;
-      end = end > toDeleteList.size() ? toDeleteList.size() : end;
+    Iterator<Block> iter = toDeleteList.iterator();
+    while (iter.hasNext()) {
       writeLock();
       try {
-        for (int i = start; i < end; i++) {
-          blockManager.removeBlock(toDeleteList.get(i));
+        for (int i = 0; i < BLOCK_DELETION_INCREMENT && iter.hasNext(); i++) {
+          blockManager.removeBlock(iter.next());
         }
       } finally {
         writeUnlock();
       }
-      start = end;
     }
   }
   
@@ -6782,6 +6779,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return; // Return previous response
     }
     boolean success = false;
+    checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -6792,7 +6790,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkOwner(pc, snapshotRoot);
 
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-      List<INode> removedINodes = new ArrayList<INode>();
+      List<INode> removedINodes = new ChunkedArrayList<INode>();
       dir.writeLock();
       try {
         snapshotManager.deleteSnapshot(snapshotRoot, snapshotName,

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.Diff;
 import org.apache.hadoop.util.StringUtils;
 
@@ -707,7 +708,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     }
     
     public BlocksMapUpdateInfo() {
-      toDeleteList = new ArrayList<Block>();
+      toDeleteList = new ChunkedArrayList<Block>();
     }
     
     /**

+ 171 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ChunkedArrayList.java

@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.util;
+
+import java.util.AbstractList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+/**
+ * Simplified List implementation which stores elements as a list
+ * of chunks, each chunk having a maximum size. This improves over
+ * using an ArrayList in that creating a large list will never require
+ * a large amount of contiguous heap space -- thus reducing the likelihood
+ * of triggering a CMS compaction pause due to heap fragmentation.
+ * 
+ * The first chunks allocated are small, but each additional chunk is
+ * 50% larger than the previous, ramping up to a configurable maximum
+ * chunk size. Reasonable defaults are provided which should be a good
+ * balance between not making any large allocations while still retaining
+ * decent performance.
+ *
+ * This currently only supports a small subset of List operations --
+ * namely addition and iteration.
+ */
+@InterfaceAudience.Private
+public class ChunkedArrayList<T> extends AbstractList<T> {
+
+  /**
+   * The chunks which make up the full list.
+   */
+  private final List<List<T>> chunks = Lists.newArrayList();
+  
+  /**
+   * Cache of the last element in the 'chunks' array above.
+   * This speeds up the add operation measurably.
+   */
+  private List<T> lastChunk = null;
+
+  /**
+   * The capacity with which the last chunk was allocated.
+   */
+  private int lastChunkCapacity;
+  
+  /**
+   * The capacity of the first chunk to allocate in a cleared list.
+   */
+  private final int initialChunkCapacity;
+  
+  /**
+   * The maximum number of elements for any chunk.
+   */
+  private final int maxChunkSize;
+
+  /**
+   * Total number of elements in the list.
+   */
+  private int size;
+  
+  /**
+   * Default initial size is 6 elements, since typical minimum object
+   * size is 64 bytes, and this leaves enough space for the object
+   * header.
+   */
+  private static final int DEFAULT_INITIAL_CHUNK_CAPACITY = 6;
+  
+  /**
+   * Default max size is 8K elements - which, at 8 bytes per element
+   * should be about 64KB -- small enough to easily fit in contiguous
+   * free heap space even with a fair amount of fragmentation.
+   */
+  private static final int DEFAULT_MAX_CHUNK_SIZE = 8*1024;
+  
+
+  public ChunkedArrayList() {
+    this(DEFAULT_INITIAL_CHUNK_CAPACITY, DEFAULT_MAX_CHUNK_SIZE);
+  }
+
+  /**
+   * @param initialChunkCapacity the capacity of the first chunk to be
+   * allocated
+   * @param maxChunkSize the maximum size of any chunk allocated
+   */
+  public ChunkedArrayList(int initialChunkCapacity, int maxChunkSize) {
+    Preconditions.checkArgument(maxChunkSize >= initialChunkCapacity);
+    this.initialChunkCapacity = initialChunkCapacity;
+    this.maxChunkSize = maxChunkSize;
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return Iterables.concat(chunks).iterator();
+  }
+
+  @Override
+  public boolean add(T e) {
+    if (lastChunk == null) {
+      addChunk(initialChunkCapacity);
+    } else if (lastChunk.size() >= lastChunkCapacity) {
+      int newCapacity = lastChunkCapacity + (lastChunkCapacity >> 1);
+      addChunk(Math.min(newCapacity, maxChunkSize));
+    }
+    size++;
+    return lastChunk.add(e);
+  }
+
+  @Override
+  public void clear() {
+    chunks.clear();
+    lastChunk = null;
+    lastChunkCapacity = 0;
+    size = 0;
+  }
+  
+  private void addChunk(int capacity) {
+    lastChunk = Lists.newArrayListWithCapacity(capacity);
+    chunks.add(lastChunk);
+    lastChunkCapacity = capacity;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return size == 0;
+  }
+
+  @Override
+  public int size() {
+    return size;
+  }
+  
+  @VisibleForTesting
+  int getNumChunks() {
+    return chunks.size();
+  }
+  
+  @VisibleForTesting
+  int getMaxChunkSize() {
+    int size = 0;
+    for (List<T> chunk : chunks) {
+      size = Math.max(size, chunk.size());
+    }
+    return size;
+  }
+
+  @Override
+  public T get(int arg0) {
+    throw new UnsupportedOperationException(
+        this.getClass().getName() + " does not support random access");
+  }
+}

+ 14 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLUtils.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java

@@ -30,19 +30,27 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.LimitedPrivate({"HDFS"})
 @InterfaceStability.Unstable
-public class URLUtils {
+public class URLConnectionFactory {
   /**
    * Timeout for socket connects and reads
    */
-  public static int SOCKET_TIMEOUT = 1*60*1000; // 1 minute
+  public final static int DEFAULT_SOCKET_TIMEOUT = 1*60*1000; // 1 minute
 
+  public static final URLConnectionFactory DEFAULT_CONNECTION_FACTORY = new URLConnectionFactory(DEFAULT_SOCKET_TIMEOUT);
+  
+  private int socketTimeout;
+
+  public URLConnectionFactory(int socketTimeout) {
+    this.socketTimeout = socketTimeout;
+  }
+  
   /**
    * Opens a url with read and connect timeouts
    * @param url to open
    * @return URLConnection
    * @throws IOException
    */
-  public static URLConnection openConnection(URL url) throws IOException {
+  public URLConnection openConnection(URL url) throws IOException {
     URLConnection connection = url.openConnection();
     setTimeouts(connection);
     return connection;    
@@ -53,8 +61,8 @@ public class URLUtils {
    * 
    * @param connection URLConnection to set
    */
-  static void setTimeouts(URLConnection connection) {
-    connection.setConnectTimeout(SOCKET_TIMEOUT);
-    connection.setReadTimeout(SOCKET_TIMEOUT);
+  public void setTimeouts(URLConnection connection) {
+    connection.setConnectTimeout(socketTimeout);
+    connection.setReadTimeout(socketTimeout);
   }
 }

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -119,13 +119,15 @@ public class WebHdfsFileSystem extends FileSystem
 
   /** SPNEGO authenticator */
   private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
+  /** Default connection factory may be overriden in tests to use smaller timeout values */
+  URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
   /** Configures connections for AuthenticatedURL */
-  private static final ConnectionConfigurator CONN_CONFIGURATOR =
+  private final ConnectionConfigurator CONN_CONFIGURATOR =
     new ConnectionConfigurator() {
       @Override
       public HttpURLConnection configure(HttpURLConnection conn)
           throws IOException {
-        URLUtils.setTimeouts(conn);
+        connectionFactory.setTimeouts(conn);
         return conn;
       }
     };
@@ -479,10 +481,9 @@ public class WebHdfsFileSystem extends FileSystem
           final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
           conn = new AuthenticatedURL(AUTH, CONN_CONFIGURATOR).openConnection(
             url, authToken);
-          URLUtils.setTimeouts(conn);
         } else {
           LOG.debug("open URL connection");
-          conn = (HttpURLConnection)URLUtils.openConnection(url);
+          conn = (HttpURLConnection)connectionFactory.openConnection(url);
         }
       } catch (AuthenticationException e) {
         throw new IOException(e);
@@ -577,7 +578,7 @@ public class WebHdfsFileSystem extends FileSystem
       checkRetry = false;
       
       //Step 2) Submit another Http request with the URL from the Location header with data.
-      conn = (HttpURLConnection)URLUtils.openConnection(new URL(redirect));
+      conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect));
       conn.setRequestProperty("Content-Type", MediaType.APPLICATION_OCTET_STREAM);
       conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
       connect();
@@ -600,7 +601,7 @@ public class WebHdfsFileSystem extends FileSystem
           disconnect();
   
           checkRetry = false;
-          conn = (HttpURLConnection)URLUtils.openConnection(new URL(redirect));
+          conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect));
           connect();
         }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/fs/TestFiRename.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.fs;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.EnumSet;
 
@@ -27,6 +28,7 @@ import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -56,8 +58,7 @@ public class TestFiRename {
   private static String addChild = "";
   private static byte[] data = { 0 };
   
-  private static String TEST_ROOT_DIR = 
-    System.getProperty("test.build.data", "/tmp") + "/test";
+  private static String TEST_ROOT_DIR = PathUtils.getTestDirName(TestFiRename.class);
   
   private static Configuration CONF = new Configuration();
   static {

+ 0 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java

@@ -59,9 +59,6 @@ public class TestHDFSFileContextMainOperations extends
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
-    // Make defaultWorkingDirectory snapshottable to enable 
-    // testGlobStatusFilterWithHiddenPathTrivialFilter
-    cluster.getFileSystem().allowSnapshot(defaultWorkingDirectory);
   }
 
   private static void restartCluster() throws IOException, LoginException {
@@ -76,9 +73,6 @@ public class TestHDFSFileContextMainOperations extends
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
-    // Make defaultWorkingDirectory snapshottable to enable 
-    // testGlobStatusFilterWithHiddenPathTrivialFilter
-    cluster.getFileSystem().allowSnapshot(defaultWorkingDirectory);
   }
       
   @AfterClass
@@ -98,11 +92,6 @@ public class TestHDFSFileContextMainOperations extends
     super.tearDown();
   }
 
-  @Override
-  protected Path getHiddenPathForTest() {
-    return new Path(defaultWorkingDirectory, ".snapshot");
-  }
-  
   @Override
   protected Path getDefaultWorkingDirectory() {
     return defaultWorkingDirectory;

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs;
 
+import java.io.File;
 import static org.junit.Assert.fail;
 
 import java.io.FileNotFoundException;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -48,6 +50,7 @@ import org.junit.Test;
  * underlying file system as Hdfs.
  */
 public class TestResolveHdfsSymlink {
+  private static File TEST_ROOT_DIR = PathUtils.getTestDir(TestResolveHdfsSymlink.class);
   private static MiniDFSCluster cluster = null;
 
   @BeforeClass
@@ -80,12 +83,12 @@ public class TestResolveHdfsSymlink {
         .getUri());
 
     Path alphaLocalPath = new Path(fcLocal.getDefaultFileSystem().getUri()
-        .toString(), "/tmp/alpha");
+        .toString(), new File(TEST_ROOT_DIR, "alpha").getAbsolutePath());
     DFSTestUtil.createFile(FileSystem.getLocal(conf), alphaLocalPath, 16,
         (short) 1, 2);
 
     Path linkTarget = new Path(fcLocal.getDefaultFileSystem().getUri()
-        .toString(), "/tmp");
+        .toString(), TEST_ROOT_DIR.getAbsolutePath());
     Path hdfsLink = new Path(fcHdfs.getDefaultFileSystem().getUri().toString(),
         "/tmp/link");
     fcHdfs.createSymlink(linkTarget, hdfsLink, true);

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java

@@ -31,6 +31,7 @@ import java.net.URL;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -38,6 +39,8 @@ import org.junit.Test;
  */
 public class TestUrlStreamHandler {
 
+  private static File TEST_ROOT_DIR = PathUtils.getTestDir(TestUrlStreamHandler.class);
+    
   /**
    * Test opening and reading from an InputStream through a hdfs:// URL.
    * <p>
@@ -111,13 +114,12 @@ public class TestUrlStreamHandler {
     Configuration conf = new HdfsConfiguration();
 
     // Locate the test temporary directory.
-    File tmpDir = new File(conf.get("hadoop.tmp.dir"));
-    if (!tmpDir.exists()) {
-      if (!tmpDir.mkdirs())
-        throw new IOException("Cannot create temporary directory: " + tmpDir);
+    if (!TEST_ROOT_DIR.exists()) {
+      if (!TEST_ROOT_DIR.mkdirs())
+        throw new IOException("Cannot create temporary directory: " + TEST_ROOT_DIR);
     }
 
-    File tmpFile = new File(tmpDir, "thefile");
+    File tmpFile = new File(TEST_ROOT_DIR, "thefile");
     URI uri = tmpFile.toURI();
 
     FileSystem fs = FileSystem.get(uri, conf);

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -39,8 +40,7 @@ import org.junit.Test;
 public class TestLoadGenerator extends Configured implements Tool {
   private static final Configuration CONF = new HdfsConfiguration();
   private static final int DEFAULT_BLOCK_SIZE = 10;
-  private static final String OUT_DIR = 
-    System.getProperty("test.build.data","build/test/data");
+  private static final File OUT_DIR = PathUtils.getTestDir(TestLoadGenerator.class);
   private static final File DIR_STRUCTURE_FILE = 
     new File(OUT_DIR, StructureGenerator.DIR_STRUCTURE_FILE_NAME);
   private static final File FILE_STRUCTURE_FILE =
@@ -65,7 +65,7 @@ public class TestLoadGenerator extends Configured implements Tool {
     StructureGenerator sg = new StructureGenerator();
     String[] args = new String[]{"-maxDepth", "2", "-minWidth", "1",
         "-maxWidth", "2", "-numOfFiles", "2",
-        "-avgFileSize", "1", "-outDir", OUT_DIR, "-seed", "1"};
+        "-avgFileSize", "1", "-outDir", OUT_DIR.getAbsolutePath(), "-seed", "1"};
     
     final int MAX_DEPTH = 1;
     final int MIN_WIDTH = 3;
@@ -133,8 +133,7 @@ public class TestLoadGenerator extends Configured implements Tool {
   public void testLoadGenerator() throws Exception {
     final String TEST_SPACE_ROOT = "/test";
 
-    final String SCRIPT_TEST_DIR = new File(System.getProperty("test.build.data",
-    "/tmp")).getAbsolutePath();
+    final String SCRIPT_TEST_DIR = OUT_DIR.getAbsolutePath();
     String script = SCRIPT_TEST_DIR + "/" + "loadgenscript";
     String script2 = SCRIPT_TEST_DIR + "/" + "loadgenscript2";
     File scriptFile1 = new File(script);
@@ -156,7 +155,7 @@ public class TestLoadGenerator extends Configured implements Tool {
     try {
       DataGenerator dg = new DataGenerator();
       dg.setConf(CONF);
-      String [] args = new String[] {"-inDir", OUT_DIR, "-root", TEST_SPACE_ROOT};
+      String [] args = new String[] {"-inDir", OUT_DIR.getAbsolutePath(), "-root", TEST_SPACE_ROOT};
       assertEquals(0, dg.run(args));
 
       final int READ_PROBABILITY = 1;

+ 22 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -1407,6 +1407,13 @@ public class MiniDFSCluster {
    * Shutdown all the nodes in the cluster.
    */
   public void shutdown() {
+      shutdown(false);
+  }
+    
+  /**
+   * Shutdown all the nodes in the cluster.
+   */
+  public void shutdown(boolean deleteDfsDir) {
     LOG.info("Shutting down the Mini HDFS Cluster");
     if (checkExitOnShutdown)  {
       if (ExitUtil.terminateCalled()) {
@@ -1426,6 +1433,11 @@ public class MiniDFSCluster {
         nameNode = null;
       }
     }
+    if (deleteDfsDir) {
+        base_dir.delete();
+    } else {
+        base_dir.deleteOnExit();
+    }
   }
   
   /**
@@ -2118,7 +2130,7 @@ public class MiniDFSCluster {
    * <li><base directory>/data/data<2*dnIndex + 1></li>
    * <li><base directory>/data/data<2*dnIndex + 2></li>
    * </ol>
-   * 
+   *
    * @param dnIndex datanode index (starts from 0)
    * @param dirIndex directory index (0 or 1). Index 0 provides access to the
    *          first storage directory. Index 1 provides access to the second
@@ -2149,7 +2161,7 @@ public class MiniDFSCluster {
   public static String getDNCurrentDir(File storageDir) {
     return storageDir + "/" + Storage.STORAGE_DIR_CURRENT + "/";
   }
-  
+
   /**
    * Get directory corresponding to block pool directory in the datanode
    * @param storageDir the storage directory of a datanode.
@@ -2255,7 +2267,7 @@ public class MiniDFSCluster {
     }
     return null;
   }
-
+  
   /**
    * Get the block metadata file for a block from a given datanode
    * 
@@ -2343,14 +2355,17 @@ public class MiniDFSCluster {
     } else {
       if (checkDataNodeAddrConfig) {
         conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
-        conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
-        conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       } else {
         conf.set(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
-        conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
-        conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       }
     }
+    if (checkDataNodeAddrConfig) {
+      conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+      conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
+    } else {
+      conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+      conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
+    }
   }
   
   private void addToFile(String p, String address) throws IOException {

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java

@@ -71,10 +71,6 @@ public class TestClientReportBadBlock {
 
   @Before
   public void startUpCluster() throws IOException {
-    if (System.getProperty("test.build.data") == null) { // to allow test to be
-      // run outside of Ant
-      System.setProperty("test.build.data", "build/test/data");
-    }
     // disable block scanner
     conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); 
     

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

@@ -20,9 +20,6 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
 import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;

+ 5 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
@@ -61,9 +62,7 @@ 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(' ', '+');
+  static final String TEST_ROOT_DIR = PathUtils.getTestDirName(TestDFSShell.class);
 
   static Path writeFile(FileSystem fs, Path f) throws IOException {
     DataOutputStream out = fs.create(f);
@@ -482,12 +481,11 @@ public class TestDFSShell {
     Configuration dstConf = new HdfsConfiguration();
     MiniDFSCluster srcCluster =  null;
     MiniDFSCluster dstCluster = null;
-    String bak = System.getProperty("test.build.data");
+    File bak = new File(PathUtils.getTestDir(getClass()), "dfs_tmp_uri");
+    bak.mkdirs();
     try{
       srcCluster = new MiniDFSCluster.Builder(srcConf).numDataNodes(2).build();
-      File nameDir = new File(new File(bak), "dfs_tmp_uri/");
-      nameDir.mkdirs();
-      System.setProperty("test.build.data", nameDir.toString());
+      dstConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, bak.getAbsolutePath());
       dstCluster = new MiniDFSCluster.Builder(dstConf).numDataNodes(2).build();
       FileSystem srcFs = srcCluster.getFileSystem();
       FileSystem dstFs = dstCluster.getFileSystem();
@@ -559,7 +557,6 @@ public class TestDFSShell {
       ret = ToolRunner.run(shell, argv);
       assertEquals("default works for rm/rmr", 0, ret);
     } finally {
-      System.setProperty("test.build.data", bak);
       if (null != srcCluster) {
         srcCluster.shutdown();
       }

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

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -75,7 +76,7 @@ public class TestDecommission {
     // Set up the hosts/exclude files.
     localFileSys = FileSystem.getLocal(conf);
     Path workingDir = localFileSys.getWorkingDirectory();
-    Path dir = new Path(workingDir, System.getProperty("test.build.data", "target/test/data") + "/work-dir/decommission");
+    Path dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission");
     hostsFile = new Path(dir, "hosts");
     excludeFile = new Path(dir, "exclude");
     

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -25,6 +25,8 @@ import static org.junit.Assert.fail;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
+
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
@@ -73,6 +75,9 @@ public class TestDistributedFileSystem {
     HdfsConfiguration conf;
     if (noXmlDefaults) {
        conf = new HdfsConfiguration(false);
+       String namenodeDir = new File(MiniDFSCluster.getBaseDirectory(), "name").getAbsolutePath();
+       conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, namenodeDir);
+       conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, namenodeDir);     
     } else {
        conf = new HdfsConfiguration();
     }

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

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -260,7 +261,7 @@ public class TestFSInputChecker {
     // create a file and verify that checksum corruption results in 
     // a checksum exception on LocalFS
     
-    String dir = System.getProperty("test.build.data", ".");
+    String dir = PathUtils.getTestDirName(getClass());
     Path file = new Path(dir + "/corruption-test.dat");
     Path crcFile = new Path(dir + "/.corruption-test.dat.crc");
     

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

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -145,7 +146,7 @@ public class TestFileAppendRestart {
 
     String tarFile = System.getProperty("test.cache.data", "build/test/cache")
       + "/" + HADOOP_23_BROKEN_APPEND_TGZ;
-    String testDir = System.getProperty("test.build.data", "build/test/data");
+    String testDir = PathUtils.getTestDirName(getClass());
     File dfsDir = new File(testDir, "image-with-buggy-append");
     if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) {
       throw new IOException("Could not delete dfs directory '" + dfsDir + "'");

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

@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -95,7 +96,7 @@ public class TestFileCorruption {
   @Test
   public void testLocalFileCorruption() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    Path file = new Path(System.getProperty("test.build.data"), "corruptFile");
+    Path file = new Path(PathUtils.getTestDirName(getClass()), "corruptFile");
     FileSystem fs = FileSystem.getLocal(conf);
     DataOutputStream dos = fs.create(file);
     dos.writeBytes("original bytes");

+ 7 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.server.namenode.BackupNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -53,6 +54,9 @@ public class TestHDFSServerPorts {
   
   // reset default 0.0.0.0 addresses in order to avoid IPv6 problem
   static final String THIS_HOST = getFullHostName() + ":0";
+  
+  private static final File TEST_DATA_DIR = PathUtils.getTestDir(TestHDFSServerPorts.class);
+  
   static {
     DefaultMetricsSystem.setMiniClusterMode(true);
   }
@@ -81,13 +85,6 @@ public class TestHDFSServerPorts {
     }
   }
   
-  /**
-   * Get base directory these tests should run in.
-   */
-  private String getTestingDir() {
-    return System.getProperty("test.build.data", "build/test/data");
-  }
-  
   public NameNode startNameNode() throws IOException {
     return startNameNode(false);
   }
@@ -95,8 +92,7 @@ public class TestHDFSServerPorts {
    * Start the namenode.
    */
   public NameNode startNameNode(boolean withService) throws IOException {
-    String dataDir = getTestingDir();
-    hdfsDir = new File(dataDir, "dfs");
+    hdfsDir = new File(TEST_DATA_DIR, "dfs");
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
@@ -119,9 +115,8 @@ public class TestHDFSServerPorts {
    * Start the BackupNode
    */
   public BackupNode startBackupNode(Configuration conf) throws IOException {
-    String dataDir = getTestingDir();
     // Set up testing environment directories
-    hdfsDir = new File(dataDir, "backupNode");
+    hdfsDir = new File(TEST_DATA_DIR, "backupNode");
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
@@ -150,8 +145,7 @@ public class TestHDFSServerPorts {
    */
   public DataNode startDataNode(int index, Configuration config) 
   throws IOException {
-    String dataDir = getTestingDir();
-    File dataNodeDir = new File(dataDir, "data-" + index);
+    File dataNodeDir = new File(TEST_DATA_DIR, "data-" + index);
     config.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataNodeDir.getPath());
 
     String[] args = new String[] {};

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java

@@ -33,16 +33,11 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.web.URLUtils;
-import org.junit.BeforeClass;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.junit.Test;
 
 public class TestHftpURLTimeouts {
-  @BeforeClass
-  public static void setup() {
-    URLUtils.SOCKET_TIMEOUT = 5;
-  }
-  
+
   @Test
   public void testHftpSocketTimeout() throws Exception {
     Configuration conf = new Configuration();
@@ -51,9 +46,11 @@ public class TestHftpURLTimeouts {
         InetAddress.getByName(null).getHostAddress(),
         socket.getLocalPort(),
         null, null, null);
-    boolean timedout = false;
 
     HftpFileSystem fs = (HftpFileSystem)FileSystem.get(uri, conf);
+    fs.connectionFactory = new URLConnectionFactory(5);
+
+    boolean timedout = false;
     try {
       HttpURLConnection conn = fs.openConnection("/", "");
       timedout = false;
@@ -69,6 +66,7 @@ public class TestHftpURLTimeouts {
       assertTrue("read timedout", timedout);
       assertTrue("connect timedout", checkConnectTimeout(fs, false));
     } finally {
+      fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
       fs.close();
     }
   }
@@ -84,6 +82,8 @@ public class TestHftpURLTimeouts {
     boolean timedout = false;
 
     HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf);
+    fs.connectionFactory = new URLConnectionFactory(5);
+    
     try {
       HttpURLConnection conn = null;
       timedout = false;
@@ -100,6 +100,7 @@ public class TestHftpURLTimeouts {
       assertTrue("ssl read connect timedout", timedout);
       assertTrue("connect timedout", checkConnectTimeout(fs, true));
     } finally {
+      fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
       fs.close();
     }
   }
@@ -121,7 +122,7 @@ public class TestHftpURLTimeouts {
           // https will get a read timeout due to SSL negotiation, but
           // a normal http will not, so need to ignore SSL read timeouts
           // until a connect timeout occurs
-          if (!(ignoreReadTimeout && message.equals("Read timed out"))) {
+          if (!(ignoreReadTimeout && "Read timed out".equals(message))) {
             timedout = true;
             assertEquals("connect timed out", message);
           }

+ 5 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java

@@ -25,8 +25,8 @@ import static org.junit.Assume.assumeTrue;
 import java.io.File;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.junit.After;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -43,20 +43,10 @@ public class TestMiniDFSCluster {
   private static final String CLUSTER_3 = "cluster3";
   private static final String CLUSTER_4 = "cluster4";
   private static final String CLUSTER_5 = "cluster5";
-  protected String testDataPath;
-  protected File testDataDir;
+  protected File testDataPath;
   @Before
   public void setUp() {
-    testDataPath = System.getProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
-        "build/test/data");
-    testDataDir = new File(new File(testDataPath).getParentFile(),
-                           "miniclusters");
-
-
-  }
-  @After
-  public void tearDown() {
-    System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA, testDataPath);
+    testDataPath = new File(PathUtils.getTestDir(getClass()), "miniclusters");
   }
 
   /**
@@ -120,7 +110,7 @@ public class TestMiniDFSCluster {
     MiniDFSCluster cluster4 = new MiniDFSCluster.Builder(conf).build();
     try {
       DistributedFileSystem dfs = (DistributedFileSystem) cluster4.getFileSystem();
-      dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       cluster4.shutdown();
     } finally {
       while(cluster4.isClusterUp()){

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

@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -322,7 +323,7 @@ public class TestPersistBlocks {
         
     String tarFile = System.getProperty("test.cache.data", "build/test/cache")
       + "/" + HADOOP_1_0_MULTIBLOCK_TGZ;
-    String testDir = System.getProperty("test.build.data", "build/test/data");
+    String testDir = PathUtils.getTestDirName(getClass());
     File dfsDir = new File(testDir, "image-1.0");
     if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) {
       throw new IOException("Could not delete dfs directory '" + dfsDir + "'");

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -174,6 +174,7 @@ public class TestNNWithQJM {
   public void testMismatchedNNIsRejected() throws Exception {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
+    String defaultEditsDir = conf.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         mjc.getQuorumJournalURI("myjournal").toString());
     
@@ -187,7 +188,7 @@ public class TestNNWithQJM {
     
     // Reformat just the on-disk portion
     Configuration onDiskOnly = new Configuration(conf);
-    onDiskOnly.unset(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
+    onDiskOnly.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, defaultEditsDir);
     NameNode.format(onDiskOnly);
 
     // Start the NN - should fail because the JNs are still formatted

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Shell;
 import org.junit.After;
 import org.junit.Before;
@@ -61,13 +62,13 @@ public class TestJournalNode {
   private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
       12345, "mycluster", "my-bp", 0L);
 
+  private static File TEST_BUILD_DATA = PathUtils.getTestDir(TestJournalNode.class);
+
   private JournalNode jn;
   private Journal journal; 
   private Configuration conf = new Configuration();
   private IPCLoggerChannel ch;
   private String journalId;
-  private File TEST_BUILD_DATA =
-      new File(System.getProperty("test.build.data", "build/test/data"));
 
   static {
     // Avoid an error when we double-initialize JvmMetrics

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java

@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.Collection;
 
 import org.apache.hadoop.conf.Configuration;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -94,8 +95,7 @@ public class TestReplicationPolicy {
 
     FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-    File baseDir = new File(System.getProperty(
-        "test.build.data", "build/test/data"), "dfs/");
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         new File(baseDir, "name").getPath());
 

+ 32 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java

@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -41,6 +40,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.PathUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestReplicationPolicyWithNodeGroup {
@@ -48,10 +50,10 @@ public class TestReplicationPolicyWithNodeGroup {
   private static final int NUM_OF_DATANODES = 8;
   private static final int NUM_OF_DATANODES_BOUNDARY = 6;
   private static final int NUM_OF_DATANODES_MORE_TARGETS = 12;
-  private static final Configuration CONF = new HdfsConfiguration();
-  private static final NetworkTopology cluster;
-  private static final NameNode namenode;
-  private static final BlockPlacementPolicy replicator;
+  private final Configuration CONF = new HdfsConfiguration();
+  private NetworkTopology cluster;
+  private NameNode namenode;
+  private BlockPlacementPolicy replicator;
   private static final String filename = "/dummyfile.txt";
 
   private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
@@ -94,27 +96,23 @@ public class TestReplicationPolicyWithNodeGroup {
   private final static DatanodeDescriptor NODE = 
       new DatanodeDescriptor(DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/d2/r4/n7"));
 
-  static {
-    try {
-      FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
-      CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-      // Set properties to make HDFS aware of NodeGroup.
-      CONF.set("dfs.block.replicator.classname", 
-          "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup");
-      CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
-          "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
-      
-      File baseDir = new File(System.getProperty(
-          "test.build.data", "build/test/data"), "dfs/");
-      CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-          new File(baseDir, "name").getPath());
-      
-      DFSTestUtil.formatNameNode(CONF);
-      namenode = new NameNode(CONF);
-    } catch (IOException e) {
-      e.printStackTrace();
-      throw (RuntimeException)new RuntimeException().initCause(e);
-    }
+  @Before
+  public void setUp() throws Exception {
+    FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
+    CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    // Set properties to make HDFS aware of NodeGroup.
+    CONF.set("dfs.block.replicator.classname", 
+        "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup");
+    CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
+        "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
+    
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicyWithNodeGroup.class);
+    
+    CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        new File(baseDir, "name").getPath());
+    
+    DFSTestUtil.formatNameNode(CONF);
+    namenode = new NameNode(CONF);
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
     replicator = bm.getBlockPlacementPolicy();
     cluster = bm.getDatanodeManager().getNetworkTopology();
@@ -125,6 +123,11 @@ public class TestReplicationPolicyWithNodeGroup {
     setupDataNodeCapacity();
   }
 
+  @After
+  public void tearDown() throws Exception {
+    namenode.stop();
+  }
+  
   private static void setupDataNodeCapacity() {
     for(int i=0; i<NUM_OF_DATANODES; i++) {
       dataNodes[i].updateHeartbeat(
@@ -636,7 +639,9 @@ public class TestReplicationPolicyWithNodeGroup {
    */
   @Test
   public void testChooseMoreTargetsThanNodeGroups() throws Exception {
-    // Cleanup nodes in previous tests
+    for(int i=0; i<NUM_OF_DATANODES; i++) {
+      cluster.remove(dataNodes[i]);
+    }
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
       DatanodeDescriptor node = dataNodesInBoundaryCase[i];
       if (cluster.contains(node)) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Before;
 import org.junit.Test;
@@ -70,8 +71,7 @@ public class TestBPOfferService {
       TestBPOfferService.class);
   private static final ExtendedBlock FAKE_BLOCK =
     new ExtendedBlock(FAKE_BPID, 12345L);
-  private static final String TEST_BUILD_DATA = System.getProperty(
-    "test.build.data", "build/test/data");
+  private static final File TEST_BUILD_DATA = PathUtils.getTestDir(TestBPOfferService.class);
 
   static {
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);

+ 13 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.TestGenericJournalConf.DummyJournalManager;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -54,37 +55,34 @@ public class TestAllowFormat {
   public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
   private static final Log LOG =
     LogFactory.getLog(TestAllowFormat.class.getName());
+  private static final File DFS_BASE_DIR = new File(PathUtils.getTestDir(TestAllowFormat.class), "dfs");
   private static Configuration config;
   private static MiniDFSCluster cluster = null;
-  private static File hdfsDir=null;
 
   @BeforeClass
   public static void setUp() throws Exception {
     config = new Configuration();
-    String baseDir = System.getProperty("test.build.data", "build/test/data");
-
-    hdfsDir = new File(baseDir, "dfs");
-    if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
-      throw new IOException("Could not delete hdfs directory '" + hdfsDir +
+    if ( DFS_BASE_DIR.exists() && !FileUtil.fullyDelete(DFS_BASE_DIR) ) {
+      throw new IOException("Could not delete hdfs directory '" + DFS_BASE_DIR +
                             "'");
     }
-
+    
     // Test has multiple name directories.
     // Format should not really prompt us if one of the directories exist,
     // but is empty. So in case the test hangs on an input, it means something
     // could be wrong in the format prompting code. (HDFS-1636)
-    LOG.info("hdfsdir is " + hdfsDir.getAbsolutePath());
-    File nameDir1 = new File(hdfsDir, "name1");
-    File nameDir2 = new File(hdfsDir, "name2");
+    LOG.info("hdfsdir is " + DFS_BASE_DIR.getAbsolutePath());
+    File nameDir1 = new File(DFS_BASE_DIR, "name1");
+    File nameDir2 = new File(DFS_BASE_DIR, "name2");
 
     // To test multiple directory handling, we pre-create one of the name directories.
     nameDir1.mkdirs();
 
     // Set multiple name directories.
     config.set(DFS_NAMENODE_NAME_DIR_KEY, nameDir1.getPath() + "," + nameDir2.getPath());
-    config.set(DFS_DATANODE_DATA_DIR_KEY, new File(hdfsDir, "data").getPath());
+    config.set(DFS_DATANODE_DATA_DIR_KEY, new File(DFS_BASE_DIR, "data").getPath());
 
-    config.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(hdfsDir, "secondary").getPath());
+    config.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(DFS_BASE_DIR, "secondary").getPath());
 
     FileSystem.setDefaultUri(config, "hdfs://"+NAME_NODE_HOST + "0");
   }
@@ -99,9 +97,9 @@ public class TestAllowFormat {
       LOG.info("Stopping mini cluster");
     }
     
-    if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
+    if ( DFS_BASE_DIR.exists() && !FileUtil.fullyDelete(DFS_BASE_DIR) ) {
       throw new IOException("Could not delete hdfs directory in tearDown '"
-                            + hdfsDir + "'");
+                            + DFS_BASE_DIR + "'");
     }	
   }
 
@@ -170,7 +168,7 @@ public class TestAllowFormat {
     HATestUtil.setFailoverConfigurations(conf, logicalName, nnAddr1, nnAddr2);
 
     conf.set(DFS_NAMENODE_NAME_DIR_KEY,
-        new File(hdfsDir, "name").getAbsolutePath());
+        new File(DFS_BASE_DIR, "name").getAbsolutePath());
     conf.setBoolean(DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY, true);
     conf.set(DFSUtil.addKeySuffixes(
         DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX, "dummy"),

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

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -58,8 +59,7 @@ import org.junit.Test;
  * A JUnit test that audit logs are generated
  */
 public class TestAuditLogs {
-  static final String auditLogFile = System.getProperty("test.build.dir",
-      "build/test") + "/audit.log";
+  static final String auditLogFile = PathUtils.getTestDirName(TestAuditLogs.class) + "/TestAuditLogs-audit.log";
   
   // Pattern for: 
   // allowed=(true|false) ugi=name ip=/address cmd={cmd} src={path} dst=null perm=null

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java

@@ -100,6 +100,8 @@ public class TestBackupNode {
         "${" + DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + "}");
     c.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY,
         "127.0.0.1:0");
+    c.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
+            "127.0.0.1:0");
 
     BackupNode bn = (BackupNode)NameNode.createNameNode(
         new String[]{startupOpt.getName()}, c);

+ 9 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -75,6 +75,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.apache.hadoop.util.StringUtils;
@@ -193,7 +194,7 @@ public class TestCheckpoint {
     ArrayList<URI> fsImageDirs = new ArrayList<URI>();
     ArrayList<URI> editsDirs = new ArrayList<URI>();
     File filePath =
-      new File(System.getProperty("test.build.data","/tmp"), "storageDirToCheck");
+      new File(PathUtils.getTestDir(getClass()), "storageDirToCheck");
     assertTrue("Couldn't create directory storageDirToCheck",
                filePath.exists() || filePath.mkdirs());
     fsImageDirs.add(filePath.toURI());
@@ -1911,9 +1912,11 @@ public class TestCheckpoint {
       }
       
       // Start a new NN with the same host/port.
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
-          .nameNodePort(origPort).nameNodeHttpPort(origHttpPort).format(true)
-          .build();
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
+          .nameNodePort(origPort)
+          .nameNodeHttpPort(origHttpPort)
+          .format(true).build();
 
       try {
         secondary.doCheckpoint();
@@ -2135,7 +2138,8 @@ public class TestCheckpoint {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
     
     try {
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
           .format(true).build();
       FileSystem fs = cluster.getFileSystem();
       secondary = startSecondaryNameNode(conf);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
@@ -72,7 +73,7 @@ public class TestClusterId {
   public void setUp() throws IOException {
     ExitUtil.disableSystemExit();
 
-    String baseDir = System.getProperty("test.build.data", "build/test/data");
+    String baseDir = PathUtils.getTestDirName(getClass());
 
     hdfsDir = new File(baseDir, "dfs/name");
     if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir)) {

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -96,9 +97,8 @@ public class TestEditLog {
   static final int NUM_TRANSACTIONS = 100;
   static final int NUM_THREADS = 100;
   
-  static final File TEST_DIR = new File(
-    System.getProperty("test.build.data","build/test/data"));
-
+  static final File TEST_DIR = PathUtils.getTestDir(TestEditLog.class);
+  
   /** An edits log with 3 edits from 0.20 - the result of
    * a fresh namesystem followed by hadoop fs -touchz /myfile */
   static final byte[] HADOOP20_SOME_EDITS =
@@ -569,6 +569,7 @@ public class TestEditLog {
       fail("should not be able to start");
     } catch (IOException e) {
       // expected
+      assertNotNull("Cause of exception should be ChecksumException", e.getCause());
       assertEquals("Cause of exception should be ChecksumException",
           ChecksumException.class, e.getCause().getClass());
     }

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

@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.After;
@@ -34,8 +35,7 @@ import org.junit.Test;
  * Test the EditLogFileOutputStream
  */
 public class TestEditLogFileOutputStream {
-  private final static File TEST_DIR =
-      new File(System.getProperty("test.build.data", "/tmp"));
+  private final static File TEST_DIR = PathUtils.getTestDir(TestEditLogFileOutputStream.class);
   private static final File TEST_EDITS =
       new File(TEST_DIR, "testEditLogFileOutput.log");
   final static int MIN_PREALLOCATION_LENGTH =

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

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
 import org.junit.Before;
@@ -192,8 +193,7 @@ public class TestEditLogJournalFailures {
     Configuration conf = new HdfsConfiguration();
     String[] nameDirs = new String[4];
     for (int i = 0; i < nameDirs.length; i++) {
-      File nameDir = new File(System.getProperty("test.build.data"),
-          "name-dir" + i);
+      File nameDir = new File(PathUtils.getTestDir(getClass()), "name-dir" + i);
       nameDir.mkdirs();
       nameDirs[i] = nameDir.getAbsolutePath();
     }

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

@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -60,8 +61,7 @@ public class TestFSEditLogLoader {
     ((Log4JLogger)FSEditLogLoader.LOG).getLogger().setLevel(Level.ALL);
   }
   
-  private static final File TEST_DIR = new File(
-      System.getProperty("test.build.data","build/test/data"));
+  private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
 
   private static final int NUM_DATA_NODES = 0;
   

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -91,7 +91,7 @@ import static org.mockito.Mockito.*;
  */
 public class TestFsck {
   static final String auditLogFile = System.getProperty("test.build.dir",
-      "build/test") + "/audit.log";
+      "build/test") + "/TestFsck-audit.log";
   
   // Pattern for: 
   // allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null
@@ -159,7 +159,8 @@ public class TestFsck {
       cluster.shutdown();
       
       // restart the cluster; bring up namenode but not the data nodes
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0).format(false).build();
       outStr = runFsck(conf, 1, true, "/");
       // expect the result is corrupt
       assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -60,7 +61,7 @@ public class TestNameEditsConfigs {
 
   short replication = 3;
   private File base_dir = new File(
-      System.getProperty("test.build.data", "build/test/data"), "dfs/");
+      PathUtils.getTestDir(TestNameEditsConfigs.class), "dfs");
 
   @Before
   public void setUp() throws IOException {
@@ -68,7 +69,7 @@ public class TestNameEditsConfigs {
       throw new IOException("Cannot remove directory " + base_dir);
     }
   }
-
+  
   void checkImageAndEditsFilesExistence(File dir, 
                                         boolean shouldHaveImages,
                                         boolean shouldHaveEdits)

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

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 
@@ -57,8 +58,7 @@ import com.google.common.collect.Sets;
 public class TestNameNodeRecovery {
   private static final Log LOG = LogFactory.getLog(TestNameNodeRecovery.class);
   private static StartupOption recoverStartOpt = StartupOption.RECOVER;
-  private static final File TEST_DIR = new File(
-      System.getProperty("test.build.data","build/test/data"));
+  private static final File TEST_DIR = PathUtils.getTestDir(TestNameNodeRecovery.class);
 
   static {
     recoverStartOpt.setForce(MetaRecoveryContext.FORCE_ALL);

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java

@@ -33,12 +33,14 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NameNodeResourceMonitor;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker.CheckedVolume;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 public class TestNameNodeResourceChecker {
+  private final static File BASE_DIR = PathUtils.getTestDir(TestNameNodeResourceChecker.class);
   private Configuration conf;
   private File baseDir;
   private File nameDir;
@@ -46,8 +48,7 @@ public class TestNameNodeResourceChecker {
   @Before
   public void setUp () throws IOException {
     conf = new Configuration();
-    baseDir = new File(System.getProperty("test.build.data"));
-    nameDir = new File(baseDir, "resource-check-name-dir");
+    nameDir = new File(BASE_DIR, "resource-check-name-dir");
     nameDir.mkdirs();
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath());
   }
@@ -141,8 +142,8 @@ public class TestNameNodeResourceChecker {
   @Test
   public void testChecking2NameDirsOnOneVolume() throws IOException {
     Configuration conf = new Configuration();
-    File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1");
-    File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2");
+    File nameDir1 = new File(BASE_DIR, "name-dir1");
+    File nameDir2 = new File(BASE_DIR, "name-dir2");
     nameDir1.mkdirs();
     nameDir2.mkdirs();
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
@@ -162,7 +163,7 @@ public class TestNameNodeResourceChecker {
   @Test
   public void testCheckingExtraVolumes() throws IOException {
     Configuration conf = new Configuration();
-    File nameDir = new File(System.getProperty("test.build.data"), "name-dir");
+    File nameDir = new File(BASE_DIR, "name-dir");
     nameDir.mkdirs();
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, nameDir.getAbsolutePath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_KEY, nameDir.getAbsolutePath());
@@ -182,8 +183,8 @@ public class TestNameNodeResourceChecker {
   @Test
   public void testLowResourceVolumePolicy() throws IOException, URISyntaxException {
     Configuration conf = new Configuration();
-    File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1");
-    File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2");
+    File nameDir1 = new File(BASE_DIR, "name-dir1");
+    File nameDir2 = new File(BASE_DIR, "name-dir2");
     nameDir1.mkdirs();
     nameDir2.mkdirs();
     

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

@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.junit.After;
@@ -404,8 +405,7 @@ public class TestStartup {
     Configuration conf = new Configuration();
     FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
-    File base_dir = new File(System.getProperty(
-        "test.build.data", "build/test/data"), "dfs/");
+    File base_dir = new File(PathUtils.getTestDir(getClass()), "dfs/");
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         new File(base_dir, "name").getPath());
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

@@ -391,7 +391,8 @@ public class TestStorageRestore {
         (new File(path3, "current").getAbsolutePath()) : path3.toString();
 
     try {
-      cluster = new MiniDFSCluster.Builder(config).numDataNodes(0)
+      cluster = new MiniDFSCluster.Builder(config)
+          .numDataNodes(0)
           .manageNameDfsDirs(false).build();
       cluster.waitActive();
 

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

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -48,8 +49,7 @@ import com.google.common.collect.ImmutableList;
 
 public class TestTransferFsImage {
 
-  private static final File TEST_DIR = new File(
-      System.getProperty("test.build.data","build/test/data"));
+  private static final File TEST_DIR = PathUtils.getTestDir(TestTransferFsImage.class);
 
   /**
    * Regression test for HDFS-1997. Test that, if an exception

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java

@@ -56,8 +56,8 @@ public class TestBootstrapStandby {
 
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(20001))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(20002)));
     
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java

@@ -70,13 +70,13 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
     
-    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn1", 10003);
-    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", 10004);
+    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn1", 10023);
+    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", 10024);
 
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
     .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10021))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10022)));
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)
         .numDataNodes(0)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java

@@ -124,8 +124,8 @@ public class TestEditLogTailer {
     // Have to specify IPC ports so the NNs can talk to each other.
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10031))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10032)));
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java

@@ -76,8 +76,8 @@ public class TestFailureToReadEdits {
     
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10041))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10042)));
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)
       .numDataNodes(0)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java

@@ -52,8 +52,8 @@ public class TestHAFsck {
     // need some HTTP ports
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ha-nn-uri-0")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10051))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10052)));
     
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

@@ -89,8 +89,8 @@ public class TestStandbyCheckpoints {
 
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10061))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10062)));
     
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
 import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
 import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer.Flags;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -53,7 +54,7 @@ public class TestOfflineEditsViewer {
   }
 
   private static String buildDir =
-    System.getProperty("test.build.data", "build/test/data");
+    PathUtils.getTestDirName(TestOfflineEditsViewer.class);
 
   private static String cacheDir =
     System.getProperty("test.cache.data", "build/test/cache");

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java

@@ -27,6 +27,7 @@ import java.io.FileReader;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -34,7 +35,7 @@ import org.junit.Test;
  * on predetermined inputs
  */
 public class TestDelimitedImageVisitor {
-  private static String ROOT = System.getProperty("test.build.data","/tmp");
+  private static String ROOT = PathUtils.getTestDirName(TestDelimitedImageVisitor.class);
   private static final String delim = "--";
   
   // Record an element in the visitor and build the expected line in the output

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -88,8 +89,7 @@ public class TestOfflineImageViewer {
   final static HashMap<String, FileStatus> writtenFiles = 
       new HashMap<String, FileStatus>();
   
-  private static String ROOT = System.getProperty("test.build.data",
-                                                  "build/test/data");
+  private static String ROOT = PathUtils.getTestDirName(TestOfflineImageViewer.class);
   
   // Create a populated namespace for later testing.  Save its contents to a
   // data structure and store its fsimage location.

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestAtomicFileOutputStream.java

@@ -30,6 +30,7 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -40,10 +41,7 @@ public class TestAtomicFileOutputStream {
   private static final String TEST_STRING = "hello world";
   private static final String TEST_STRING_2 = "goodbye world";
 
-  private static File BASE_DIR = new File(
-      System.getProperty("test.build.data", "build/test/data"));
-  private static File TEST_DIR = new File(BASE_DIR,
-      TestAtomicFileOutputStream.class.getName());
+  private static File TEST_DIR = PathUtils.getTestDir(TestAtomicFileOutputStream.class);
   
   private static File DST_FILE = new File(TEST_DIR, "test.txt");
   

+ 93 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestChunkedArrayList.java

@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.util;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+
+import org.junit.Test;
+
+import com.google.common.base.Stopwatch;
+
+public class TestChunkedArrayList {
+
+  @Test
+  public void testBasics() {
+    final int N_ELEMS = 100000;
+    ChunkedArrayList<Integer> l = new ChunkedArrayList<Integer>();
+    assertTrue(l.isEmpty());
+    // Insert a bunch of elements.
+    for (int i = 0; i < N_ELEMS; i++) {
+      l.add(i);
+    }
+    assertFalse(l.isEmpty());
+    assertEquals(N_ELEMS, l.size());
+
+    // Check that it got chunked.
+    assertTrue(l.getNumChunks() > 10);
+    assertEquals(8192, l.getMaxChunkSize());
+  }
+  
+  @Test
+  public void testIterator() {
+    ChunkedArrayList<Integer> l = new ChunkedArrayList<Integer>();
+    for (int i = 0; i < 30000; i++) {
+      l.add(i);
+    }
+    
+    int i = 0;
+    for (int fromList : l) {
+      assertEquals(i, fromList);
+      i++;
+    }
+  }
+  
+  @Test
+  public void testPerformance() {
+    String obj = "hello world";
+    
+    final int numElems = 1000000;
+    final int numTrials = 5;
+    
+    for (int trial = 0; trial < numTrials; trial++) {
+      System.gc();
+      {
+        ArrayList<String> arrayList = new ArrayList<String>();
+        Stopwatch sw = new Stopwatch();
+        sw.start();
+        for (int i = 0; i < numElems; i++) {
+          arrayList.add(obj);
+        }
+        System.out.println("       ArrayList " + sw.elapsedMillis());
+      }
+      
+      // test ChunkedArrayList
+      System.gc();
+      {
+        ChunkedArrayList<String> chunkedList = new ChunkedArrayList<String>();
+        Stopwatch sw = new Stopwatch();
+        sw.start();
+        for (int i = 0; i < numElems; i++) {
+          chunkedList.add(obj);
+        }
+        System.out.println("ChunkedArrayList " + sw.elapsedMillis());
+      }
+    }
+  }
+}

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestMD5FileUtils.java

@@ -29,14 +29,12 @@ import java.io.IOException;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestMD5FileUtils {
-  private static final File TEST_DIR_ROOT = new File(
-      System.getProperty("test.build.data","build/test/data"));
-  private static final File TEST_DIR = new File(TEST_DIR_ROOT,
-      "TestMD5FileUtils");
+  private static final File TEST_DIR = PathUtils.getTestDir(TestMD5FileUtils.class);
   private static final File TEST_FILE = new File(TEST_DIR,
       "testMd5File.dat");
   

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java

@@ -58,7 +58,6 @@ public class TestWebHdfsTimeouts {
 
   private static final int CLIENTS_TO_CONSUME_BACKLOG = 100;
   private static final int CONNECTION_BACKLOG = 1;
-  private static final int INITIAL_SOCKET_TIMEOUT = URLUtils.SOCKET_TIMEOUT;
   private static final int SHORT_SOCKET_TIMEOUT = 5;
   private static final int TEST_TIMEOUT = 10000;
 
@@ -67,20 +66,22 @@ public class TestWebHdfsTimeouts {
   private InetSocketAddress nnHttpAddress;
   private ServerSocket serverSocket;
   private Thread serverThread;
+  private URLConnectionFactory connectionFactory = new URLConnectionFactory(SHORT_SOCKET_TIMEOUT);
 
   @Before
   public void setUp() throws Exception {
-    URLUtils.SOCKET_TIMEOUT = SHORT_SOCKET_TIMEOUT;
     Configuration conf = WebHdfsTestUtil.createConf();
     nnHttpAddress = NameNode.getHttpAddress(conf);
     serverSocket = new ServerSocket(nnHttpAddress.getPort(), CONNECTION_BACKLOG);
     fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
+    fs.connectionFactory = connectionFactory;
     clients = new ArrayList<SocketChannel>();
     serverThread = null;
   }
 
   @After
   public void tearDown() throws Exception {
+    fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
     IOUtils.cleanup(LOG, clients.toArray(new SocketChannel[clients.size()]));
     IOUtils.cleanup(LOG, fs);
     if (serverSocket != null) {
@@ -240,7 +241,7 @@ public class TestWebHdfsTimeouts {
    */
   private void startSingleTemporaryRedirectResponseThread(
       final boolean consumeConnectionBacklog) {
-    URLUtils.SOCKET_TIMEOUT = INITIAL_SOCKET_TIMEOUT;
+    fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
     serverThread = new Thread() {
       @Override
       public void run() {
@@ -254,7 +255,7 @@ public class TestWebHdfsTimeouts {
           clientSocket = serverSocket.accept();
 
           // Immediately setup conditions for subsequent connections.
-          URLUtils.SOCKET_TIMEOUT = SHORT_SOCKET_TIMEOUT;
+          fs.connectionFactory = connectionFactory;
           if (consumeConnectionBacklog) {
             consumeConnectionBacklog();
           }

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.test;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+
+public class PathUtils {
+
+  public static Path getTestPath(Class<?> caller) {
+    return getTestPath(caller, true);
+  }
+
+  public static Path getTestPath(Class<?> caller, boolean create) {
+    return new Path(getTestDirName(caller));
+  }
+
+  public static File getTestDir(Class<?> caller) {
+    return getTestDir(caller, true);
+  }
+  
+  public static File getTestDir(Class<?> caller, boolean create) {
+    File dir = new File(System.getProperty("test.build.data", "/tmp"), caller.getSimpleName());
+    if (create) {
+      dir.mkdirs();
+    }
+    return dir;
+  }
+
+  public static String getTestDirName(Class<?> caller) {
+    return getTestDirName(caller, true);
+  }
+  
+  public static String getTestDirName(Class<?> caller, boolean create) {
+    return getTestDir(caller, create).getAbsolutePath();
+  }
+    
+}

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

@@ -253,6 +253,9 @@ Release 2.1.1-beta - UNRELEASED
 
     MAPREDUCE-5475. MRClientService does not verify ACLs properly (jlowe)
 
+    MAPREDUCE-5414. TestTaskAttempt fails in JDK7 with NPE (Nemon Lou via 
+    devaraj)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java

@@ -343,7 +343,7 @@ public class TestTaskAttempt{
     TaskAttemptImpl taImpl =
       new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
           splits, jobConf, taListener,
-          mock(Token.class), new Credentials(),
+          new Token(), new Credentials(),
           new SystemClock(), null);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
@@ -399,7 +399,7 @@ public class TestTaskAttempt{
     TaskAttemptImpl taImpl =
       new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
           splits, jobConf, taListener,
-          mock(Token.class), new Credentials(),
+          new Token(), new Credentials(),
           new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.2", 0);
@@ -456,7 +456,7 @@ public class TestTaskAttempt{
     TaskAttemptImpl taImpl =
       new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
           splits, jobConf, taListener,
-          mock(Token.class), new Credentials(),
+          new Token(), new Credentials(),
           new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
@@ -516,7 +516,7 @@ public class TestTaskAttempt{
     TaskAttemptImpl taImpl =
       new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
           splits, jobConf, taListener,
-          mock(Token.class), new Credentials(),
+          new Token(), new Credentials(),
           new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
@@ -582,7 +582,7 @@ public class TestTaskAttempt{
 
     TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
         jobFile, 1, splits, jobConf, taListener,
-        mock(Token.class), new Credentials(), new SystemClock(), appCtx);
+        new Token(), new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
@@ -631,7 +631,7 @@ public class TestTaskAttempt{
 
     TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
         jobFile, 1, splits, jobConf, taListener,
-        mock(Token.class), new Credentials(), new SystemClock(), appCtx);
+        new Token(), new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
     ContainerId contId = ContainerId.newInstance(appAttemptId, 3);

+ 2 - 20
hadoop-project/pom.xml

@@ -713,7 +713,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.12.3</version>
+          <version>2.16</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -827,7 +827,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <forkMode>always</forkMode>
+          <reuseForks>false</reuseForks>
           <forkedProcessTimeoutInSeconds>900</forkedProcessTimeoutInSeconds>
           <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError</argLine>
           <environmentVariables>
@@ -1002,23 +1002,5 @@
         </plugins>
       </build>
     </profile>
-    <!-- Copied into specific modules supporting parallel testing. Will be uncommented as soon as all modules support this.
-    <profile>
-      <id>parallel-tests</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-surefire-plugin</artifactId>
-            <configuration>
-              <forkMode>perthread</forkMode>
-              <threadCount>${testsThreadCount}</threadCount>
-              <parallel>classes</parallel>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    -->
   </profiles>
 </project>

+ 29 - 3
hadoop-yarn-project/CHANGES.txt

@@ -27,14 +27,13 @@ Release 2.3.0 - UNRELEASED
   IMPROVEMENTS
 
     YARN-905. Add state filters to nodes CLI (Wei Yan via Sandy Ryza)
+    YARN-1098. Separate out RM services into Always On and Active (Karthik
+    Kambatla via bikas)
 
   OPTIMIZATIONS
 
   BUG FIXES
 
-    YARN-758. Augment MockNM to use multiple cores (Karthik Kambatla via
-    Sandy Ryza)
-
     YARN-1060. Two tests in TestFairScheduler are missing @Test annotation
     (Niranjan Singh via Sandy Ryza)
 
@@ -89,6 +88,16 @@ Release 2.1.1-beta - UNRELEASED
     YARN-1065. NM should provide AuxillaryService data to the container (Xuan
     Gong via bikas)
 
+    YARN-758. Augment MockNM to use multiple cores (Karthik Kambatla via
+    Sandy Ryza)
+
+    YARN-696. Changed RMWebservice apps call to take in multiple application
+    states. (Trevor Lorimer via vinodkv)
+
+    YARN-910. Augmented auxiliary services to listen for container starts and
+    completions in addition to application events. (Alejandro Abdelnur via
+    vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -156,6 +165,23 @@ Release 2.1.1-beta - UNRELEASED
     need more than a node's total capability were incorrectly allocated on that
     node causing apps to hang. (Omkar Vinit Joshi via vinodkv)
 
+    YARN-1107. Fixed a bug in ResourceManager because of which RM in secure mode
+    fails to restart. (Omkar Vinit Joshi via vinodkv)
+
+    YARN-1049. ContainerExistStatus should define a status for preempted 
+    containers. (tucu)
+
+    YARN-1144. Unmanaged AMs registering a tracking URI should not be 
+    proxy-fied. (tucu)
+
+    YARN-1152. Fixed a bug in ResourceManager that was causing clients to get
+    invalid client token key errors when an appliation is about to finish.
+    (Jason Lowe via vinodkv)
+
+    YARN-292. Fixed FifoScheduler and FairScheduler to make their applications
+    data structures thread safe to avoid RM crashing with
+    ArrayIndexOutOfBoundsException. (Zhijie Shen via vinodkv)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java

@@ -41,4 +41,9 @@ public class ContainerExitStatus {
    * threshold number of the nodemanager-log-directories become bad.
    */
   public static final int DISKS_FAILED = -101;
+
+  /**
+   * Containers preempted by the framework.
+   */
+  public static final int PREEMPTED = -102;
 }

+ 24 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/AuxiliaryService.java

@@ -79,4 +79,27 @@ public abstract class AuxiliaryService extends AbstractService {
    *         applications.
    */
   public abstract ByteBuffer getMetaData();
-}
+
+  /**
+   * A new container is started on this NodeManager. This is a signal to
+   * this {@link AuxiliaryService} about the container initialization.
+   * This method is called when the NodeManager receives the container launch
+   * command from the ApplicationMaster and before the container process is 
+   * launched.
+   *
+   * @param initContainerContext context for the container's initialization
+   */
+  public void initializeContainer(ContainerInitializationContext
+      initContainerContext) {
+  }
+
+  /**
+   * A container is finishing on this NodeManager. This is a signal to this
+   * {@link AuxiliaryService} about the same.
+   *
+   * @param stopContainerContext context for the container termination
+   */
+  public void stopContainer(ContainerTerminationContext stopContainerContext) {
+  }
+
+}

+ 75 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java

@@ -0,0 +1,75 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * Base context class for {@link AuxiliaryService} initializing and stopping a
+ * container.
+ */
+@Public
+@Evolving
+public class ContainerContext {
+  private final String user;
+  private final ContainerId containerId;
+  private final Resource resource;
+
+  @Private
+  @Unstable
+  public ContainerContext(String user, ContainerId containerId,
+      Resource resource) {
+    this.user = user;
+    this.containerId = containerId;
+    this.resource = resource;
+  }
+
+  /**
+   * Get user of the container being initialized or stopped.
+   *
+   * @return the user
+   */
+  public String getUser() {
+    return user;
+  }
+
+  /**
+   * Get {@link ContainerId} of the container being initialized or stopped.
+   *
+   * @return the container ID
+   */
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+
+  /**
+   * Get {@link Resource} the resource capability allocated to the container
+   * being initialized or stopped.
+   *
+   * @return the resource capability.
+   */
+  public Resource getResource() {
+    return resource;
+  }
+}

+ 44 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java

@@ -0,0 +1,44 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * Initialization context for {@link AuxiliaryService} when starting a
+ * container.
+ * 
+ */
+@Public
+@Evolving
+public class ContainerInitializationContext extends ContainerContext {
+
+  @Private
+  @Unstable
+  public ContainerInitializationContext(String user, ContainerId containerId,
+      Resource resource) {
+    super(user, containerId, resource);
+  }
+
+}

+ 44 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java

@@ -0,0 +1,44 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * Termination context for {@link AuxiliaryService} when stopping a
+ * container.
+ * 
+ */
+@Public
+@Evolving
+public class ContainerTerminationContext extends ContainerContext {
+
+  @Private
+  @Unstable
+  public ContainerTerminationContext(String user, ContainerId containerId,
+      Resource resource) {
+    super(user, containerId, resource);
+  }
+
+}

+ 17 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java

@@ -37,6 +37,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
+import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
+import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 
 public class AuxServices extends AbstractService
     implements ServiceStateChangeListener, EventHandler<AuxServicesEvent> {
@@ -178,7 +180,21 @@ public class AuxServices extends AbstractService
           .getApplicationID()));
       }
       break;
-    default:
+    case CONTAINER_INIT:
+      for (AuxiliaryService serv : serviceMap.values()) {
+        serv.initializeContainer(new ContainerInitializationContext(
+            event.getUser(), event.getContainer().getContainerId(),
+            event.getContainer().getResource()));
+      }
+      break;
+    case CONTAINER_STOP:
+      for (AuxiliaryService serv : serviceMap.values()) {
+        serv.stopContainer(new ContainerTerminationContext(
+            event.getUser(), event.getContainer().getContainerId(),
+            event.getContainer().getResource()));
+      }
+      break;
+      default:
       throw new RuntimeException("Unknown type: " + event.getType());
     }
   }

+ 19 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEvent.java

@@ -21,7 +21,10 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+    .Container;
 
 public class AuxServicesEvent extends AbstractEvent<AuxServicesEventType> {
 
@@ -29,18 +32,30 @@ public class AuxServicesEvent extends AbstractEvent<AuxServicesEventType> {
   private final String serviceId;
   private final ByteBuffer serviceData;
   private final ApplicationId appId;
+  private final Container container;
 
   public AuxServicesEvent(AuxServicesEventType eventType, ApplicationId appId) {
     this(eventType, null, appId, null, null);
   }
 
+  public AuxServicesEvent(AuxServicesEventType eventType, Container container) {
+    this(eventType, null, container.getContainerId().getApplicationAttemptId()
+        .getApplicationId(), null, null, container);
+  }
+
   public AuxServicesEvent(AuxServicesEventType eventType, String user,
       ApplicationId appId, String serviceId, ByteBuffer serviceData) {
+    this(eventType, user, appId, serviceId, serviceData, null);
+  }
+    public AuxServicesEvent(AuxServicesEventType eventType, String user,
+      ApplicationId appId, String serviceId, ByteBuffer serviceData,
+        Container container) {
     super(eventType);
     this.user = user;
     this.appId = appId;
     this.serviceId = serviceId;
     this.serviceData = serviceData;
+    this.container = container;
   }
 
   public String getServiceID() {
@@ -59,4 +74,8 @@ public class AuxServicesEvent extends AbstractEvent<AuxServicesEventType> {
     return appId;
   }
 
+  public Container getContainer() {
+    return container;
+  }
+
 }

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEventType.java

@@ -20,5 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 public enum AuxServicesEventType {
   APPLICATION_INIT,
-  APPLICATION_STOP
+  APPLICATION_STOP,
+  CONTAINER_INIT,
+  CONTAINER_STOP
 }

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -503,6 +503,9 @@ public class ContainerImpl implements Container {
       final ContainerLaunchContext ctxt = container.launchContext;
       container.metrics.initingContainer();
 
+      container.dispatcher.getEventHandler().handle(new AuxServicesEvent
+          (AuxServicesEventType.CONTAINER_INIT, container));
+
       // Inform the AuxServices about the opaque serviceData
       Map<String,ByteBuffer> csd = ctxt.getServiceData();
       if (csd != null) {
@@ -820,8 +823,16 @@ public class ContainerImpl implements Container {
   static class ContainerDoneTransition implements
       SingleArcTransition<ContainerImpl, ContainerEvent> {
     @Override
+    @SuppressWarnings("unchecked")
     public void transition(ContainerImpl container, ContainerEvent event) {
       container.finished();
+      //if the current state is NEW it means the CONTAINER_INIT was never 
+      // sent for the event, thus no need to send the CONTAINER_STOP
+      if (container.getCurrentState() 
+          != org.apache.hadoop.yarn.api.records.ContainerState.NEW) {
+        container.dispatcher.getEventHandler().handle(new AuxServicesEvent
+            (AuxServicesEventType.CONTAINER_STOP, container));
+      }
     }
   }
 

+ 59 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java

@@ -22,6 +22,7 @@ import static org.apache.hadoop.service.Service.STATE.INITED;
 import static org.apache.hadoop.service.Service.STATE.STARTED;
 import static org.apache.hadoop.service.Service.STATE.STOPPED;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -34,11 +35,21 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
+import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
+import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+    .Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+    .ContainerImpl;
 import org.junit.Test;
 
 public class TestAuxServices {
@@ -52,8 +63,10 @@ public class TestAuxServices {
     private int remaining_stop;
     private ByteBuffer meta = null;
     private ArrayList<Integer> stoppedApps;
+    private ContainerId containerId;
+    private Resource resource;
 
-    LightService(String name, char idef, int expected_appId) {
+         LightService(String name, char idef, int expected_appId) {
       this(name, idef, expected_appId, null);
     } 
     LightService(String name, char idef, int expected_appId, ByteBuffer meta) {
@@ -95,7 +108,22 @@ public class TestAuxServices {
     public ByteBuffer getMetaData() {
       return meta;
     }
-  }
+
+    @Override
+    public void initializeContainer(
+        ContainerInitializationContext initContainerContext) {
+      containerId = initContainerContext.getContainerId();
+      resource = initContainerContext.getResource();
+    }
+
+    @Override
+    public void stopContainer(
+        ContainerTerminationContext stopContainerContext) {
+      containerId = stopContainerContext.getContainerId();
+      resource = stopContainerContext.getResource();
+    }
+
+ }
 
   static class ServiceA extends LightService {
     public ServiceA() { 
@@ -142,6 +170,35 @@ public class TestAuxServices {
       assertEquals("app not properly stopped", 1, appIds.size());
       assertTrue("wrong app stopped", appIds.contains((Integer)66));
     }
+
+    for (AuxiliaryService serv : servs) {
+      assertNull(((LightService) serv).containerId);
+      assertNull(((LightService) serv).resource);
+    }
+
+
+    ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId1, 1);
+    ContainerTokenIdentifier cti = new ContainerTokenIdentifier(
+        ContainerId.newInstance(attemptId, 1), "", "",
+        Resource.newInstance(1, 1), 0,0,0);
+    Container container = new ContainerImpl(null, null, null, null, null, cti);
+    ContainerId containerId = container.getContainerId();
+    Resource resource = container.getResource();
+    event = new AuxServicesEvent(AuxServicesEventType.CONTAINER_INIT,container);
+    aux.handle(event);
+    for (AuxiliaryService serv : servs) {
+      assertEquals(containerId, ((LightService) serv).containerId);
+      assertEquals(resource, ((LightService) serv).resource);
+      ((LightService) serv).containerId = null;
+      ((LightService) serv).resource = null;
+    }
+
+    event = new AuxServicesEvent(AuxServicesEventType.CONTAINER_STOP, container);
+    aux.handle(event);
+    for (AuxiliaryService serv : servs) {
+      assertEquals(containerId, ((LightService) serv).containerId);
+      assertEquals(resource, ((LightService) serv).resource);
+    }
   }
 
   @Test

部分文件因为文件数量过多而无法显示