فهرست منبع

Merge branch 'trunk' into HDFS-7240

Anu Engineer 9 سال پیش
والد
کامیت
b5f772abda
100فایلهای تغییر یافته به همراه1332 افزوده شده و 453 حذف شده
  1. 20 20
      LICENSE.txt
  2. 1 1
      NOTICE.txt
  3. 53 16
      dev-support/bin/create-release
  4. 5 0
      dev-support/docker/Dockerfile
  5. 4 0
      hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java
  6. 2 2
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
  7. 2 2
      hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md
  8. 2 1
      hadoop-common-project/hadoop-common/pom.xml
  9. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  10. 21 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
  11. 72 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PrintableString.java
  12. 6 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
  13. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
  14. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  15. 97 17
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
  16. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
  17. 13 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  18. 2 1
      hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
  19. 78 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellList.java
  20. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java
  21. 87 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPrintableString.java
  22. 26 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/package-info.java
  23. 5 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
  24. 4 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  25. 3 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
  26. 55 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
  27. 32 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
  28. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
  29. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
  30. 5 1
      hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java
  32. 3 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
  33. 2 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java
  34. 3 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java
  35. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java
  36. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  37. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  38. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
  39. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  40. 18 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  41. 11 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
  42. 27 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
  43. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  44. 67 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
  45. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
  46. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md
  48. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
  49. 5 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
  50. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java
  51. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java
  52. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  53. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java
  54. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  55. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  56. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  57. 11 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
  58. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
  59. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java
  60. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java
  61. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
  62. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
  63. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
  64. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
  65. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  66. 83 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java
  67. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
  68. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
  69. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java
  70. 18 47
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
  71. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
  72. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  73. 78 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  74. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java
  75. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
  76. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
  77. 125 36
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java
  78. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  79. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  80. 11 51
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
  81. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
  82. 25 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java
  83. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
  84. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
  85. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
  86. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/PathUtils.java
  87. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
  88. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  89. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  90. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
  91. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  92. 33 33
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
  93. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
  94. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  95. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
  96. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
  97. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  98. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
  99. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
  100. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java

+ 20 - 20
LICENSE.txt

@@ -207,7 +207,7 @@ APACHE HADOOP SUBCOMPONENTS:
 The Apache Hadoop project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these
 subcomponents is subject to the terms and conditions of the following
-licenses. 
+licenses.
 
 For the org.apache.hadoop.util.bloom.* classes:
 
@@ -216,30 +216,30 @@ For the org.apache.hadoop.util.bloom.* classes:
  * Copyright (c) 2005, European Commission project OneLab under contract
  * 034819 (http://www.one-lab.org)
  * All rights reserved.
- * Redistribution and use in source and binary forms, with or 
- * without modification, are permitted provided that the following 
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
  * conditions are met:
- *  - Redistributions of source code must retain the above copyright 
+ *  - Redistributions of source code must retain the above copyright
  *    notice, this list of conditions and the following disclaimer.
- *  - Redistributions in binary form must reproduce the above copyright 
- *    notice, this list of conditions and the following disclaimer in 
+ *  - Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in
  *    the documentation and/or other materials provided with the distribution.
  *  - Neither the name of the University Catholique de Louvain - UCL
- *    nor the names of its contributors may be used to endorse or 
- *    promote products derived from this software without specific prior 
+ *    nor the names of its contributors may be used to endorse or
+ *    promote products derived from this software without specific prior
  *    written permission.
- *    
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
- * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
- * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
- * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
- * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
- * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
- * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
- * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
- * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
- * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
  * POSSIBILITY OF SUCH DAMAGE.
  */
 

+ 1 - 1
NOTICE.txt

@@ -280,4 +280,4 @@ which has the following notices:
   Dumbster SMTP test server
     Copyright 2004 Jason Paul Kitchen
   TypeUtil.java
-    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams
+    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams

+ 53 - 16
dev-support/bin/create-release

@@ -240,6 +240,8 @@ function set_defaults
   OSNAME=$(uname -s)
 
   PUBKEYFILE="https://dist.apache.org/repos/dist/release/hadoop/common/KEYS"
+
+  SIGN=false
 }
 
 function startgpgagent
@@ -247,11 +249,23 @@ function startgpgagent
   if [[ "${SIGN}" = true ]]; then
     if [[ -n "${GPGAGENT}" && -z "${GPG_AGENT_INFO}" ]]; then
       echo "starting gpg agent"
-      touch "${LOGDIR}/gpgagent.conf"
+      echo "default-cache-ttl 7200" > "${LOGDIR}/gpgagent.conf"
+      # shellcheck disable=2046
       eval $("${GPGAGENT}" --daemon \
         --options "${LOGDIR}/gpgagent.conf" \
-        --log-file=${LOGDIR}/create-release-gpgagent.log)
-      GPGAGENTPID=$(echo ${GPG_AGENT_INFO} | cut -f 2 -d:)
+        --log-file="${LOGDIR}/create-release-gpgagent.log")
+      GPGAGENTPID=$(echo "${GPG_AGENT_INFO}" | cut -f 2 -d:)
+    fi
+
+    if [[ -n "${GPG_AGENT_INFO}" ]]; then
+      echo "Warming the gpg-agent cache prior to calling maven"
+      # warm the agent's cache:
+      touch "${LOGDIR}/warm"
+      ${GPG} --use-agent --armor --output "${LOGDIR}/warm.asc" --detach-sig "${LOGDIR}/warm"
+      rm "${LOGDIR}/warm.asc" "${LOGDIR}/warm"
+    else
+      SIGN=false
+      hadoop_error "ERROR: Unable to launch or acquire gpg-agent. Disable signing."
     fi
   fi
 }
@@ -259,7 +273,7 @@ function startgpgagent
 function stopgpgagent
 {
   if [[ -n "${GPGAGENTPID}" ]]; then
-    kill ${GPGAGENTPID}
+    kill "${GPGAGENTPID}"
   fi
 }
 
@@ -273,7 +287,7 @@ function usage
   echo "--mvncache=[path]       Path to the maven cache to use"
   echo "--native                Also build the native components"
   echo "--rc-label=[label]      Add this label to the builds"
-  echo "--sign                  Use .gnupg dir to sign the jars"
+  echo "--sign                  Use .gnupg dir to sign the artifacts and jars"
   echo "--version=[version]     Use an alternative version string"
 }
 
@@ -330,6 +344,16 @@ function option_parse
     SIGN=false
   fi
 
+  if [[ "${SIGN}" = true ]]; then
+    if [[ -n "${GPG_AGENT_INFO}" ]]; then
+      echo "NOTE: Using existing gpg-agent. If the default-cache-ttl"
+      echo "is set to less than ~20 mins, maven commands will fail."
+    elif [[ -z "${GPGAGENT}" ]]; then
+      hadoop_error "ERROR: No gpg-agent. Disabling signing capability."
+      SIGN=false
+    fi
+  fi
+
   DOCKERCMD=$(command -v docker)
   if [[ "${DOCKER}" = true && -z "${DOCKERCMD}" ]]; then
       hadoop_error "ERROR: docker binary not found. Disabling docker mode."
@@ -439,6 +463,11 @@ function dockermode
     # make sure we put some space between, just in case last
     # line isn't an empty line or whatever
     printf "\n\n"
+
+    # force a new image for every run to make it easier to remove later
+    echo "LABEL org.apache.hadoop.create-release=\"cr-${RANDOM}\""
+
+    # setup ownerships, etc
     echo "RUN groupadd --non-unique -g ${group_id} ${user_name}"
     echo "RUN useradd -g ${group_id} -u ${user_id} -m ${user_name}"
     echo "RUN chown -R ${user_name} /home/${user_name}"
@@ -490,19 +519,27 @@ function makearelease
 
   big_console_header "Maven Build and Install"
 
+  if [[ "${SIGN}" = true ]]; then
+    signflags=("-Psign" "-Dgpg.useagent=true" -Dgpg.executable="${GPG}")
+  fi
+
   # Create SRC and BIN tarballs for release,
-  # Using 'install’ goal instead of 'package' so artifacts are available
-  # in the Maven local cache for the site generation
-  #
   # shellcheck disable=SC2046
   run_and_redirect "${LOGDIR}/mvn_install.log" \
-    "${MVN}" "${MVN_ARGS[@]}" install -Pdist,src \
+    "${MVN}" "${MVN_ARGS[@]}" install \
+      -Pdist,src \
+      "${signflags[@]}" \
       -DskipTests -Dtar $(hadoop_native_flags)
 
-  big_console_header "Maven Site"
-
   # Create site for release
-  run_and_redirect "${LOGDIR}/mvn_site.log" "${MVN}" "${MVN_ARGS[@]}" site site:stage -Pdist,src,releasedocs
+  # we need to do install again so that jdiff and
+  # a few other things get registered in the maven
+  # universe correctly
+  run_and_redirect "${LOGDIR}/mvn_site.log" \
+    "${MVN}" "${MVN_ARGS[@]}" install \
+      site site:stage \
+      -DskipTests \
+      -Pdist,src,releasedocs,docs
 
   big_console_header "Staging the release"
 
@@ -560,16 +597,16 @@ function signartifacts
   big_console_header "Signing the release"
 
   for i in ${ARTIFACTS_DIR}/*; do
-    gpg --use-agent --armor --output "${i}.asc" --detach-sig "${i}"
-    gpg --print-mds "${i}" > "${i}.mds"
+    ${GPG} --use-agent --armor --output "${i}.asc" --detach-sig "${i}"
+    ${GPG} --print-mds "${i}" > "${i}.mds"
     domd5 "${i}"
   done
 
   if [[ "${ASFRELEASE}" = true ]]; then
     echo "Fetching the Apache Hadoop KEYS file..."
     curl -L "${PUBKEYFILE}" -o "${BASEDIR}/target/KEYS"
-    gpg --import --trustdb "${BASEDIR}/target/testkeysdb" "${BASEDIR}/target/KEYS"
-    gpg --verify --trustdb "${BASEDIR}/target/testkeysdb" \
+    ${GPG} --import --trustdb "${BASEDIR}/target/testkeysdb" "${BASEDIR}/target/KEYS"
+    ${GPG} --verify --trustdb "${BASEDIR}/target/testkeysdb" \
       "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz.asc" \
         "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
     if [[ $? != 0 ]]; then

+ 5 - 0
dev-support/docker/Dockerfile

@@ -129,6 +129,11 @@ RUN apt-get -q install --no-install-recommends -y bats
 ####
 RUN pip install pylint
 
+####
+# Install dateutil.parser
+####
+RUN pip install python-dateutil
+
 ###
 # Avoid out of memory errors in builds
 ###

+ 4 - 0
hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java

@@ -127,6 +127,10 @@ class RootDocProcessor {
               return filter(((ClassDoc) target).constructors(true),
                   ConstructorDoc.class);
             }
+          } else {
+            if (methodName.equals("methods")) {
+              return filter(((ClassDoc) target).methods(true), MethodDoc.class);
+            }
           }
         } else if (target instanceof PackageDoc) {
           if (methodName.equals("allClasses")) {

+ 2 - 2
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java

@@ -44,14 +44,14 @@ import java.util.Map;
  * URL url = new URL("http://foo:8080/bar");
  * AuthenticatedURL.Token token = new AuthenticatedURL.Token();
  * AuthenticatedURL aUrl = new AuthenticatedURL();
- * HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+ * HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
  * ....
  * // use the 'conn' instance
  * ....
  *
  * // establishing a follow up connection using a token from the previous connection
  *
- * HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+ * HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
  * ....
  * // use the 'conn' instance
  * ....

+ 2 - 2
hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md

@@ -57,9 +57,9 @@ Use the `AuthenticatedURL` class to obtain an authenticated HTTP connection:
     URL url = new URL("http://localhost:8080/hadoop-auth/kerberos/who");
     AuthenticatedURL.Token token = new AuthenticatedURL.Token();
     ...
-    HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+    HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
     ...
-    conn = new AuthenticatedURL(url, token).openConnection();
+    conn = new AuthenticatedURL().openConnection(url, token);
     ...
 
 Building and Running the Examples

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

@@ -985,11 +985,12 @@
                     </goals>
                     <configuration>
                         <executable>${basedir}/../../dev-support/bin/releasedocmaker</executable>
-                        <workingDirectory>src/site/markdown/release/</workingDirectory>
                         <requiresOnline>true</requiresOnline>
                         <arguments>
                             <argument>--index</argument>
                             <argument>--license</argument>
+                            <argument>--outputdir</argument>
+                            <argument>${basedir}/src/site/markdown/release</argument>
                             <argument>--project</argument>
                             <argument>HADOOP</argument>
                             <argument>--project</argument>

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

@@ -98,6 +98,20 @@ public class FileUtil {
       return stat2Paths(stats);
   }
 
+  /**
+   * Register all files recursively to be deleted on exit.
+   * @param file File/directory to be deleted
+   */
+  public static void fullyDeleteOnExit(final File file) {
+    file.deleteOnExit();
+    if (file.isDirectory()) {
+      File[] files = file.listFiles();
+      for (File child : files) {
+        fullyDeleteOnExit(child);
+      }
+    }
+  }
+
   /**
    * Delete a directory and all its contents.  If
    * we return false, the directory may be partially-deleted.

+ 21 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java

@@ -48,6 +48,7 @@ class Ls extends FsCommand {
   private static final String OPTION_PATHONLY = "C";
   private static final String OPTION_DIRECTORY = "d";
   private static final String OPTION_HUMAN = "h";
+  private static final String OPTION_HIDENONPRINTABLE = "q";
   private static final String OPTION_RECURSIVE = "R";
   private static final String OPTION_REVERSE = "r";
   private static final String OPTION_MTIME = "t";
@@ -55,10 +56,11 @@ class Ls extends FsCommand {
   private static final String OPTION_SIZE = "S";
 
   public static final String NAME = "ls";
-  public static final String USAGE = "[-" + OPTION_PATHONLY + "] [-"
-      + OPTION_DIRECTORY + "] [-" + OPTION_HUMAN + "] [-" + OPTION_RECURSIVE
-      + "] [-" + OPTION_MTIME + "] [-" + OPTION_SIZE + "] [-" + OPTION_REVERSE
-      + "] [-" + OPTION_ATIME + "] [<path> ...]";
+  public static final String USAGE = "[-" + OPTION_PATHONLY + "] [-" +
+      OPTION_DIRECTORY + "] [-" + OPTION_HUMAN + "] [-" +
+      OPTION_HIDENONPRINTABLE + "] [-" + OPTION_RECURSIVE + "] [-" +
+      OPTION_MTIME + "] [-" + OPTION_SIZE + "] [-" + OPTION_REVERSE + "] [-" +
+      OPTION_ATIME + "] [<path> ...]";
 
   public static final String DESCRIPTION =
       "List the contents that match the specified file pattern. If " +
@@ -77,6 +79,8 @@ class Ls extends FsCommand {
           "  -" + OPTION_HUMAN +
           "  Formats the sizes of files in a human-readable fashion\n" +
           "      rather than a number of bytes.\n" +
+          "  -" + OPTION_HIDENONPRINTABLE +
+          "  Print ? instead of non-printable characters.\n" +
           "  -" + OPTION_RECURSIVE +
           "  Recursively list the contents of directories.\n" +
           "  -" + OPTION_MTIME +
@@ -104,6 +108,9 @@ class Ls extends FsCommand {
 
   protected boolean humanReadable = false;
 
+  /** Whether to print ? instead of non-printable characters. */
+  private boolean hideNonPrintable = false;
+
   protected Ls() {}
 
   protected Ls(Configuration conf) {
@@ -119,14 +126,16 @@ class Ls extends FsCommand {
   @Override
   protected void processOptions(LinkedList<String> args)
   throws IOException {
-    CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, OPTION_PATHONLY,
-        OPTION_DIRECTORY, OPTION_HUMAN, OPTION_RECURSIVE, OPTION_REVERSE,
+    CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE,
+        OPTION_PATHONLY, OPTION_DIRECTORY, OPTION_HUMAN,
+        OPTION_HIDENONPRINTABLE, OPTION_RECURSIVE, OPTION_REVERSE,
         OPTION_MTIME, OPTION_SIZE, OPTION_ATIME);
     cf.parse(args);
     pathOnly = cf.getOpt(OPTION_PATHONLY);
     dirRecurse = !cf.getOpt(OPTION_DIRECTORY);
     setRecursive(cf.getOpt(OPTION_RECURSIVE) && dirRecurse);
     humanReadable = cf.getOpt(OPTION_HUMAN);
+    hideNonPrintable = cf.getOpt(OPTION_HIDENONPRINTABLE);
     orderReverse = cf.getOpt(OPTION_REVERSE);
     orderTime = cf.getOpt(OPTION_MTIME);
     orderSize = !orderTime && cf.getOpt(OPTION_SIZE);
@@ -163,6 +172,11 @@ class Ls extends FsCommand {
     return this.humanReadable;
   }
 
+  @InterfaceAudience.Private
+  private boolean isHideNonPrintable() {
+    return hideNonPrintable;
+  }
+
   /**
    * Should directory contents be displayed in reverse order
    * @return true reverse order, false default order
@@ -241,7 +255,7 @@ class Ls extends FsCommand {
         dateFormat.format(new Date(isUseAtime()
             ? stat.getAccessTime()
             : stat.getModificationTime())),
-        item);
+        isHideNonPrintable() ? new PrintableString(item.toString()) : item);
     out.println(line);
   }
 

+ 72 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PrintableString.java

@@ -0,0 +1,72 @@
+/**
+ * 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.fs.shell;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The {code PrintableString} class converts any string to a printable string
+ * by replacing non-printable characters with ?.
+ *
+ * Categories of Unicode non-printable characters:
+ * <ul>
+ * <li> Control characters   (Cc)
+ * <li> Formatting Unicode   (Cf)
+ * <li> Private use Unicode  (Co)
+ * <li> Unassigned Unicode   (Cn)
+ * <li> Standalone surrogate (Unfortunately no matching Unicode category)
+ * </ul>
+ *
+ * @see Character
+ * @see <a href="http://www.unicode.org/">The Unicode Consortium</a>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+class PrintableString {
+  private static final char REPLACEMENT_CHAR = '?';
+
+  private final String printableString;
+
+  PrintableString(String rawString) {
+    StringBuilder stringBuilder = new StringBuilder(rawString.length());
+    for (int offset = 0; offset < rawString.length();) {
+      int codePoint = rawString.codePointAt(offset);
+      offset += Character.charCount(codePoint);
+
+      switch (Character.getType(codePoint)) {
+      case Character.CONTROL:     // Cc
+      case Character.FORMAT:      // Cf
+      case Character.PRIVATE_USE: // Co
+      case Character.SURROGATE:   // Cs
+      case Character.UNASSIGNED:  // Cn
+        stringBuilder.append(REPLACEMENT_CHAR);
+        break;
+      default:
+        stringBuilder.append(Character.toChars(codePoint));
+        break;
+      }
+    }
+    printableString = stringBuilder.toString();
+  }
+
+  public String toString() {
+    return printableString;
+  }
+}

+ 6 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java

@@ -207,7 +207,12 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
     // time stream might start without a leading BZ.
     final long FIRST_BZIP2_BLOCK_MARKER_POSITION =
       CBZip2InputStream.numberOfBytesTillNextMarker(seekableIn);
-    long adjStart = Math.max(0L, start - FIRST_BZIP2_BLOCK_MARKER_POSITION);
+    long adjStart = 0L;
+    if (start != 0) {
+      // Other than the first of file, the marker size is 6 bytes.
+      adjStart = Math.max(0L, start - (FIRST_BZIP2_BLOCK_MARKER_POSITION
+          - (HEADER_LEN + SUB_HEADER_LEN)));
+    }
 
     ((Seekable)seekableIn).seek(adjStart);
     SplitCompressionInputStream in =

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -297,7 +298,16 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
     log(method, isFailover, counters.failovers, retryInfo.delay, ex);
 
     if (retryInfo.delay > 0) {
-      Thread.sleep(retryInfo.delay);
+      try {
+        Thread.sleep(retryInfo.delay);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.warn("Interrupted while waiting to retry", e);
+        InterruptedIOException intIOE = new InterruptedIOException(
+            "Retry interrupted");
+        intIOE.initCause(e);
+        throw intIOE;
+      }
     }
 
     if (isFailover) {

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -2553,7 +2553,7 @@ public abstract class Server {
     this.maxDataLength = conf.getInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH,
         CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
     if (queueSizePerHandler != -1) {
-      this.maxQueueSize = queueSizePerHandler;
+      this.maxQueueSize = handlerCount * queueSizePerHandler;
     } else {
       this.maxQueueSize = handlerCount * conf.getInt(
           CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_KEY,

+ 97 - 17
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java

@@ -25,6 +25,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Hashtable;
 import java.util.List;
+import java.util.HashSet;
+import java.util.Collection;
+import java.util.Set;
 
 import javax.naming.Context;
 import javax.naming.NamingEnumeration;
@@ -66,9 +69,11 @@ import org.apache.hadoop.conf.Configuration;
  * is used for searching users or groups which returns more results than are
  * allowed by the server, an exception will be thrown.
  * 
- * The implementation also does not attempt to resolve group hierarchies. In
- * order to be considered a member of a group, the user must be an explicit
- * member in LDAP.
+ * The implementation attempts to resolve group hierarchies,
+ * to a configurable limit.
+ * If the limit is 0, in order to be considered a member of a group,
+ * the user must be an explicit member in LDAP.  Otherwise, it will traverse the
+ * group hierarchy n levels up.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
@@ -156,6 +161,13 @@ public class LdapGroupsMapping
   public static final String GROUP_NAME_ATTR_KEY = LDAP_CONFIG_PREFIX + ".search.attr.group.name";
   public static final String GROUP_NAME_ATTR_DEFAULT = "cn";
 
+  /*
+   * How many levels to traverse when checking for groups in the org hierarchy
+   */
+  public static final String GROUP_HIERARCHY_LEVELS_KEY =
+        LDAP_CONFIG_PREFIX + ".search.group.hierarchy.levels";
+  public static final int GROUP_HIERARCHY_LEVELS_DEFAULT = 0;
+
   /*
    * LDAP attribute names to use when doing posix-like lookups
    */
@@ -208,6 +220,7 @@ public class LdapGroupsMapping
   private String memberOfAttr;
   private String groupMemberAttr;
   private String groupNameAttr;
+  private int    groupHierarchyLevels;
   private String posixUidAttr;
   private String posixGidAttr;
   private boolean isPosix;
@@ -234,7 +247,7 @@ public class LdapGroupsMapping
      */
     for(int retry = 0; retry < RECONNECT_RETRY_COUNT; retry++) {
       try {
-        return doGetGroups(user);
+        return doGetGroups(user, groupHierarchyLevels);
       } catch (NamingException e) {
         LOG.warn("Failed to get groups for user " + user + " (retry=" + retry
             + ") by " + e);
@@ -324,9 +337,11 @@ public class LdapGroupsMapping
    * @return a list of strings representing group names of the user.
    * @throws NamingException if unable to find group names
    */
-  private List<String> lookupGroup(SearchResult result, DirContext c)
+  private List<String> lookupGroup(SearchResult result, DirContext c,
+      int goUpHierarchy)
       throws NamingException {
     List<String> groups = new ArrayList<String>();
+    Set<String> groupDNs = new HashSet<String>();
 
     NamingEnumeration<SearchResult> groupResults = null;
     // perform the second LDAP query
@@ -345,12 +360,14 @@ public class LdapGroupsMapping
     if (groupResults != null) {
       while (groupResults.hasMoreElements()) {
         SearchResult groupResult = groupResults.nextElement();
-        Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
-        if (groupName == null) {
-          throw new NamingException("The group object does not have " +
-              "attribute '" + groupNameAttr + "'.");
-        }
-        groups.add(groupName.get().toString());
+        getGroupNames(groupResult, groups, groupDNs, goUpHierarchy > 0);
+      }
+      if (goUpHierarchy > 0 && !isPosix) {
+        // convert groups to a set to ensure uniqueness
+        Set<String> groupset = new HashSet<String>(groups);
+        goUpGroupHierarchy(groupDNs, goUpHierarchy, groupset);
+        // convert set back to list for compatibility
+        groups = new ArrayList<String>(groupset);
       }
     }
     return groups;
@@ -369,7 +386,8 @@ public class LdapGroupsMapping
    * return an empty string array.
    * @throws NamingException if unable to get group names
    */
-  List<String> doGetGroups(String user) throws NamingException {
+  List<String> doGetGroups(String user, int goUpHierarchy)
+      throws NamingException {
     DirContext c = getDirContext();
 
     // Search for the user. We'll only ever need to look at the first result
@@ -378,7 +396,7 @@ public class LdapGroupsMapping
     // return empty list if the user can not be found.
     if (!results.hasMoreElements()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("doGetGroups(" + user + ") return no groups because the " +
+        LOG.debug("doGetGroups(" + user + ") returned no groups because the " +
             "user is not found.");
       }
       return new ArrayList<String>();
@@ -411,15 +429,76 @@ public class LdapGroupsMapping
                 "the second LDAP query using the user's DN.", e);
       }
     }
-    if (groups == null || groups.isEmpty()) {
-      groups = lookupGroup(result, c);
+    if (groups == null || groups.isEmpty() || goUpHierarchy > 0) {
+      groups = lookupGroup(result, c, goUpHierarchy);
     }
     if (LOG.isDebugEnabled()) {
-      LOG.debug("doGetGroups(" + user + ") return " + groups);
+      LOG.debug("doGetGroups(" + user + ") returned " + groups);
     }
     return groups;
   }
 
+  /* Helper function to get group name from search results.
+  */
+  void getGroupNames(SearchResult groupResult, Collection<String> groups,
+                     Collection<String> groupDNs, boolean doGetDNs)
+                     throws NamingException  {
+    Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
+    if (groupName == null) {
+      throw new NamingException("The group object does not have " +
+        "attribute '" + groupNameAttr + "'.");
+    }
+    groups.add(groupName.get().toString());
+    if (doGetDNs) {
+      groupDNs.add(groupResult.getNameInNamespace());
+    }
+  }
+
+  /* Implementation for walking up the ldap hierarchy
+   * This function will iteratively find the super-group memebership of
+   *    groups listed in groupDNs and add them to
+   * the groups set.  It will walk up the hierarchy goUpHierarchy levels.
+   * Note: This is an expensive operation and settings higher than 1
+   *    are NOT recommended as they will impact both the speed and
+   *    memory usage of all operations.
+   * The maximum time for this function will be bounded by the ldap query
+   * timeout and the number of ldap queries that it will make, which is
+   * max(Recur Depth in LDAP, goUpHierarcy) * DIRECTORY_SEARCH_TIMEOUT
+   *
+   * @param ctx - The context for contacting the ldap server
+   * @param groupDNs - the distinguished name of the groups whose parents we
+   *    want to look up
+   * @param goUpHierarchy - the number of levels to go up,
+   * @param groups - Output variable to store all groups that will be added
+  */
+  void goUpGroupHierarchy(Set<String> groupDNs,
+                          int goUpHierarchy,
+                          Set<String> groups)
+      throws NamingException {
+    if (goUpHierarchy <= 0 || groups.isEmpty()) {
+      return;
+    }
+    DirContext context = getDirContext();
+    Set<String> nextLevelGroups = new HashSet<String>();
+    StringBuilder filter = new StringBuilder();
+    filter.append("(&").append(groupSearchFilter).append("(|");
+    for (String dn : groupDNs) {
+      filter.append("(").append(groupMemberAttr).append("=")
+        .append(dn).append(")");
+    }
+    filter.append("))");
+    LOG.debug("Ldap group query string: " + filter.toString());
+    NamingEnumeration<SearchResult> groupResults =
+        context.search(baseDN,
+           filter.toString(),
+           SEARCH_CONTROLS);
+    while (groupResults.hasMoreElements()) {
+      SearchResult groupResult = groupResults.nextElement();
+      getGroupNames(groupResult, groups, nextLevelGroups, true);
+    }
+    goUpGroupHierarchy(nextLevelGroups, goUpHierarchy - 1, groups);
+  }
+
   DirContext getDirContext() throws NamingException {
     if (ctx == null) {
       // Set up the initial environment for LDAP connectivity
@@ -446,7 +525,6 @@ public class LdapGroupsMapping
 
       ctx = new InitialDirContext(env);
     }
-
     return ctx;
   }
   
@@ -513,6 +591,8 @@ public class LdapGroupsMapping
         conf.get(GROUP_MEMBERSHIP_ATTR_KEY, GROUP_MEMBERSHIP_ATTR_DEFAULT);
     groupNameAttr =
         conf.get(GROUP_NAME_ATTR_KEY, GROUP_NAME_ATTR_DEFAULT);
+    groupHierarchyLevels =
+        conf.getInt(GROUP_HIERARCHY_LEVELS_KEY, GROUP_HIERARCHY_LEVELS_DEFAULT);
     posixUidAttr =
         conf.get(POSIX_UID_ATTR_KEY, POSIX_UID_ATTR_DEFAULT);
     posixGidAttr =

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java

@@ -208,7 +208,7 @@ public class GenericOptionsParser {
    * Specify properties of each generic option
    */
   @SuppressWarnings("static-access")
-  private static Options buildGeneralOptions(Options opts) {
+  private static synchronized Options buildGeneralOptions(Options opts) {
     Option fs = OptionBuilder.withArgName("local|namenode:port")
     .hasArg()
     .withDescription("specify a namenode")

+ 13 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -315,6 +315,19 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.group.mapping.ldap.search.group.hierarchy.levels</name>
+  <value>0</value>
+  <description>
+    The number of levels to go up the group hierarchy when determining
+    which groups a user is part of. 0 Will represent checking just the
+    group that the user belongs to.  Each additional level will raise the
+    time it takes to exectue a query by at most
+    hadoop.security.group.mapping.ldap.directory.search.timeout.
+    The default will usually be appropriate for all LDAP systems.
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.group.mapping.ldap.posix.attr.uid.name</name>
   <value>uidNumber</value>

+ 2 - 1
hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md

@@ -384,13 +384,14 @@ Return usage output.
 ls
 ----
 
-Usage: `hadoop fs -ls [-C] [-d] [-h] [-R] [-t] [-S] [-r] [-u] <args> `
+Usage: `hadoop fs -ls [-C] [-d] [-h] [-q] [-R] [-t] [-S] [-r] [-u] <args> `
 
 Options:
 
 * -C: Display the paths of files and directories only.
 * -d: Directories are listed as plain files.
 * -h: Format file sizes in a human-readable fashion (eg 64.0m instead of 67108864).
+* -q: Print ? instead of non-printable characters.
 * -R: Recursively list subdirectories encountered.
 * -t: Sort output by modification time (most recent first).
 * -S: Sort output by file size.

+ 78 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellList.java

@@ -0,0 +1,78 @@
+/**
+ * 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.fs;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test FsShell -ls command.
+ */
+public class TestFsShellList {
+  private static Configuration conf;
+  private static FsShell shell;
+  private static LocalFileSystem lfs;
+  private static Path testRootDir;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = new Configuration();
+    shell = new FsShell(conf);
+    lfs = FileSystem.getLocal(conf);
+    lfs.setVerifyChecksum(true);
+    lfs.setWriteChecksum(true);
+
+    String root = System.getProperty("test.build.data", "test/build/data");
+    testRootDir = lfs.makeQualified(new Path(root, "testFsShellList"));
+    assertThat(lfs.mkdirs(testRootDir), is(true));
+  }
+
+  @AfterClass
+  public static void teardown() throws Exception {
+    lfs.delete(testRootDir, true);
+  }
+
+  private void createFile(Path filePath) throws Exception {
+    FSDataOutputStream out = lfs.create(filePath);
+    out.writeChars("I am " + filePath);
+    out.close();
+    assertThat(lfs.exists(lfs.getChecksumFile(filePath)), is(true));
+  }
+
+  @Test
+  public void testList() throws Exception {
+    createFile(new Path(testRootDir, "abc"));
+    String[] lsArgv = new String[]{"-ls", testRootDir.toString()};
+    assertThat(shell.run(lsArgv), is(0));
+
+    createFile(new Path(testRootDir, "abc\bd\tef"));
+    createFile(new Path(testRootDir, "ghi"));
+    createFile(new Path(testRootDir, "qq\r123"));
+    lsArgv = new String[]{"-ls", testRootDir.toString()};
+    assertThat(shell.run(lsArgv), is(0));
+
+    lsArgv = new String[]{"-ls", "-q", testRootDir.toString()};
+    assertThat(shell.run(lsArgv), is(0));
+  }
+}

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetSpaceUsed.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -28,8 +29,8 @@ import java.io.IOException;
 import static org.junit.Assert.*;
 
 public class TestGetSpaceUsed {
-  final static private File DIR = new File(
-      System.getProperty("test.build.data", "/tmp"), "TestGetSpaceUsed");
+  final static private File DIR =
+      GenericTestUtils.getTestDir("TestGetSpaceUsed");
 
   @Before
   public void setUp() {

+ 87 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPrintableString.java

@@ -0,0 +1,87 @@
+/**
+ * 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.fs.shell;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import org.junit.Test;
+
+/**
+ * Test {@code PrintableString} class.
+ */
+public class TestPrintableString {
+
+  private void expect(String reason, String raw, String expected) {
+    assertThat(reason, new PrintableString(raw).toString(), is(expected));
+  }
+
+  /**
+   * Test printable characters.
+   */
+  @Test
+  public void testPrintableCharacters() throws Exception {
+    // ASCII
+    expect("Should keep ASCII letter", "abcdef237", "abcdef237");
+    expect("Should keep ASCII symbol", " !\"|}~", " !\"|}~");
+
+    // Unicode BMP
+    expect("Should keep Georgian U+1050 and Box Drawing U+2533",
+        "\u1050\u2533--", "\u1050\u2533--");
+
+    // Unicode SMP
+    expect("Should keep Linear B U+10000 and Phoenician U+10900",
+        "\uD800\uDC00'''\uD802\uDD00", "\uD800\uDC00'''\uD802\uDD00");
+  }
+
+  /**
+   * Test non-printable characters.
+   */
+  @Test
+  public void testNonPrintableCharacters() throws Exception {
+    // Control characters
+    expect("Should replace single control character", "abc\rdef", "abc?def");
+    expect("Should replace multiple control characters",
+        "\babc\tdef", "?abc?def");
+    expect("Should replace all control characters", "\f\f\b\n", "????");
+    expect("Should replace mixed characters starting with a control",
+        "\027ab\0", "?ab?");
+
+    // Formatting Unicode
+    expect("Should replace Byte Order Mark", "-\uFEFF--", "-?--");
+    expect("Should replace Invisible Separator", "\u2063\t", "??");
+
+    // Private use Unicode
+    expect("Should replace private use U+E000", "\uE000", "?");
+    expect("Should replace private use U+E123 and U+F432",
+        "\uE123abc\uF432", "?abc?");
+    expect("Should replace private use in Plane 15 and 16: U+F0000 and " +
+        "U+10FFFD, but keep U+1050",
+        "x\uDB80\uDC00y\uDBFF\uDFFDz\u1050", "x?y?z\u1050");
+
+    // Unassigned Unicode
+    expect("Should replace unassigned U+30000 and U+DFFFF",
+        "-\uD880\uDC00-\uDB3F\uDFFF-", "-?-?-");
+
+    // Standalone surrogate character (not in a pair)
+    expect("Should replace standalone surrogate U+DB80", "x\uDB80yz", "x?yz");
+    expect("Should replace standalone surrogate mixed with valid pair",
+        "x\uDB80\uD802\uDD00yz", "x?\uD802\uDD00yz");
+  }
+}

+ 26 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/package-info.java

@@ -0,0 +1,26 @@
+/*
+ * 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 for {@code org.apache.hadoop.fs.shell} test classes.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.shell;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 5 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java

@@ -31,6 +31,7 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.util.Collections;
 import java.util.Map;
@@ -320,7 +321,9 @@ public class TestRetryProxy {
     futureThread.get().interrupt();
     Throwable e = future.get(1, TimeUnit.SECONDS); // should return immediately 
     assertNotNull(e);
-    assertEquals(InterruptedException.class, e.getClass());
-    assertEquals("sleep interrupted", e.getMessage());
+    assertEquals(InterruptedIOException.class, e.getClass());
+    assertEquals("Retry interrupted", e.getMessage());
+    assertEquals(InterruptedException.class, e.getCause().getClass());
+    assertEquals("sleep interrupted", e.getCause().getMessage());
   }
 }

+ 4 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -348,6 +348,10 @@ public class TestRPC extends TestRpcBase {
 
     assertEquals(3, server.getNumReaders());
     assertEquals(200, server.getMaxQueueSize());
+
+    server = newServerBuilder(conf).setQueueSizePerHandler(10)
+        .setNumHandlers(2).setVerbose(false).build();
+    assertEquals(2 * 10, server.getMaxQueueSize());
   }
 
   @Test

+ 3 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java

@@ -118,15 +118,14 @@ public class TestRollingFileSystemSink {
     diff = rfsSink.nextFlush.getTimeInMillis() - calendar.getTimeInMillis();
 
     assertTrue("The initial flush time was calculated incorrectly: " + diff,
-        (diff >= -1000L) && (diff < -900L));
+        (diff == 0L) || ((diff > -1000L) && (diff < -900L)));
 
     calendar.set(Calendar.MILLISECOND, 10);
     rfsSink.setInitialFlushTime(calendar.getTime());
     diff = rfsSink.nextFlush.getTimeInMillis() - calendar.getTimeInMillis();
 
     assertTrue("The initial flush time was calculated incorrectly: " + diff,
-        ((diff >= -10L) && (diff <= 0L) ||
-            ((diff > -1000L) && (diff < -910L))));
+        (diff >= -10L) && (diff <= 0L) || ((diff > -1000L) && (diff < -910L)));
 
     calendar.set(Calendar.SECOND, 1);
     calendar.set(Calendar.MILLISECOND, 10);
@@ -134,8 +133,7 @@ public class TestRollingFileSystemSink {
     diff = rfsSink.nextFlush.getTimeInMillis() - calendar.getTimeInMillis();
 
     assertTrue("The initial flush time was calculated incorrectly: " + diff,
-        ((diff >= -10L) && (diff <= 0L) ||
-            ((diff > -1000L) && (diff < -910L))));
+        (diff >= -10L) && (diff <= 0L) || ((diff > -1000L) && (diff < -910L)));
 
     // Now try pathological settings
     rfsSink = new RollingFileSystemSink(1000, 1000000);

+ 55 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java

@@ -39,6 +39,7 @@ import java.net.Socket;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.HashSet;
 
 import javax.naming.CommunicationException;
 import javax.naming.NamingException;
@@ -91,8 +92,23 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
     when(getContext().search(anyString(), anyString(), any(Object[].class),
         any(SearchControls.class)))
         .thenReturn(getUserNames(), getGroupNames());
-    
-    doTestGetGroups(Arrays.asList(testGroups), 2);
+    doTestGetGroups(Arrays.asList(getTestGroups()), 2);
+  }
+
+  @Test
+  public void testGetGroupsWithHierarchy() throws IOException, NamingException {
+    // The search functionality of the mock context is reused, so we will
+    // return the user NamingEnumeration first, and then the group
+    // The parent search is run once for each level, and is a different search
+    // The parent group is returned once for each group, yet the final list
+    // should be unique
+    when(getContext().search(anyString(), anyString(), any(Object[].class),
+        any(SearchControls.class)))
+        .thenReturn(getUserNames(), getGroupNames());
+    when(getContext().search(anyString(), anyString(),
+        any(SearchControls.class)))
+        .thenReturn(getParentGroupNames());
+    doTestGetGroupsWithParent(Arrays.asList(getTestParentGroups()), 2, 1);
   }
 
   @Test
@@ -104,8 +120,10 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
         .thenThrow(new CommunicationException("Connection is closed"))
         .thenReturn(getUserNames(), getGroupNames());
     
-    // Although connection is down but after reconnected it still should retrieve the result groups
-    doTestGetGroups(Arrays.asList(testGroups), 1 + 2); // 1 is the first failure call 
+    // Although connection is down but after reconnected
+    // it still should retrieve the result groups
+    // 1 is the first failure call
+    doTestGetGroups(Arrays.asList(getTestGroups()), 1 + 2);
   }
 
   @Test
@@ -139,7 +157,37 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
                                          any(Object[].class),
                                          any(SearchControls.class));
   }
-  
+
+  private void doTestGetGroupsWithParent(List<String> expectedGroups,
+      int searchTimesGroup, int searchTimesParentGroup)
+          throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    // Set the config to get parents 1 level up
+    conf.setInt(LdapGroupsMapping.GROUP_HIERARCHY_LEVELS_KEY, 1);
+
+    LdapGroupsMapping groupsMapping = getGroupsMapping();
+    groupsMapping.setConf(conf);
+    // Username is arbitrary, since the spy is mocked to respond the same,
+    // regardless of input
+    List<String> groups = groupsMapping.getGroups("some_user");
+
+    // compare lists, ignoring the order
+    Assert.assertEquals(new HashSet<String>(expectedGroups),
+        new HashSet<String>(groups));
+
+    // We should have searched for a user, and group
+    verify(getContext(), times(searchTimesGroup)).search(anyString(),
+                                         anyString(),
+                                         any(Object[].class),
+                                         any(SearchControls.class));
+    // One groups search for the parent group should have been done
+    verify(getContext(), times(searchTimesParentGroup)).search(anyString(),
+                                         anyString(),
+                                         any(SearchControls.class));
+  }
+
   @Test
   public void testExtractPassword() throws IOException {
     File testDir = GenericTestUtils.getTestDir();
@@ -246,7 +294,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
       mapping.setConf(conf);
 
       try {
-        mapping.doGetGroups("hadoop");
+        mapping.doGetGroups("hadoop", 1);
         fail("The LDAP query should have timed out!");
       } catch (NamingException ne) {
         LOG.debug("Got the exception while LDAP querying: ", ne);
@@ -302,7 +350,7 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
       mapping.setConf(conf);
 
       try {
-        mapping.doGetGroups("hadoop");
+        mapping.doGetGroups("hadoop", 1);
         fail("The LDAP query should have timed out!");
       } catch (NamingException ne) {
         LOG.debug("Got the exception while LDAP querying: ", ne);

+ 32 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java

@@ -46,13 +46,17 @@ public class TestLdapGroupsMappingBase {
   @Mock
   private NamingEnumeration<SearchResult> groupNames;
   @Mock
+  private NamingEnumeration<SearchResult> parentGroupNames;
+  @Mock
   private SearchResult userSearchResult;
   @Mock
   private Attributes attributes;
   @Spy
   private LdapGroupsMapping groupsMapping = new LdapGroupsMapping();
 
-  protected String[] testGroups = new String[] {"group1", "group2"};
+  private String[] testGroups = new String[] {"group1", "group2"};
+  private String[] testParentGroups =
+      new String[] {"group1", "group2", "group1_1"};
 
   @Before
   public void setupMocksBase() throws NamingException {
@@ -93,6 +97,24 @@ public class TestLdapGroupsMappingBase {
         thenReturn(getUserSearchResult());
 
     when(getUserSearchResult().getAttributes()).thenReturn(getAttributes());
+    // Define results for groups 1 level up
+    SearchResult parentGroupResult = mock(SearchResult.class);
+
+    // only one parent group
+    when(parentGroupNames.hasMoreElements()).thenReturn(true, false);
+    when(parentGroupNames.nextElement()).
+        thenReturn(parentGroupResult);
+
+    // Define the attribute for the parent group
+    Attribute parentGroup1Attr = new BasicAttribute("cn");
+    parentGroup1Attr.add(testParentGroups[2]);
+    Attributes parentGroup1Attrs = new BasicAttributes();
+    parentGroup1Attrs.put(parentGroup1Attr);
+
+    // attach the attributes to the result
+    when(parentGroupResult.getAttributes()).thenReturn(parentGroup1Attrs);
+    when(parentGroupResult.getNameInNamespace()).
+        thenReturn("CN=some_group,DC=test,DC=com");
   }
 
   protected DirContext getContext() {
@@ -117,4 +139,13 @@ public class TestLdapGroupsMappingBase {
   protected LdapGroupsMapping getGroupsMapping() {
     return groupsMapping;
   }
+  protected String[] getTestGroups() {
+    return testGroups;
+  }
+  protected NamingEnumeration getParentGroupNames() {
+    return parentGroupNames;
+  }
+  protected String[] getTestParentGroups() {
+    return testParentGroups;
+  }
 }

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java

@@ -69,7 +69,7 @@ public class TestLdapGroupsMappingWithPosixGroup
         any(Object[].class), any(SearchControls.class)))
         .thenReturn(getUserNames(), getGroupNames());
 
-    doTestGetGroups(Arrays.asList(testGroups), 2);
+    doTestGetGroups(Arrays.asList(getTestGroups()), 2);
   }
 
   private void doTestGetGroups(List<String> expectedGroups, int searchTimes)

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.DtFetcher;
 
 import static org.junit.Assert.assertEquals;
@@ -71,7 +72,7 @@ public class TestDtUtilShell {
 
   private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
   private final Path workDir = new Path(
-             System.getProperty("test.build.data", "/tmp"), "TestDtUtilShell");
+      GenericTestUtils.getTestDir("TestDtUtilShell").getAbsolutePath());
   private final Path tokenFile = new Path(workDir, "testPrintTokenFile");
   private final Path tokenFile2 = new Path(workDir, "testPrintTokenFile2");
   private final Path tokenLegacyFile = new Path(workDir, "testPrintTokenFile3");

+ 5 - 1
hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

@@ -54,7 +54,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-ls \[-C\] \[-d\] \[-h\] \[-R\] \[-t\] \[-S\] \[-r\] \[-u\] \[&lt;path&gt; \.\.\.\] :( |\t)*</expected-output>
+          <expected-output>^-ls \[-C\] \[-d\] \[-h\] \[-q\] \[-R\] \[-t\] \[-S\] \[-r\] \[-u\] \[&lt;path&gt; \.\.\.\] :( |\t)*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -104,6 +104,10 @@
           <type>RegexpComparator</type>
           <expected-output>^\s*-h\s+Formats the sizes of files in a human-readable fashion( )*</expected-output>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-q\s+Print \? instead of non-printable characters\.( )*</expected-output>
+        </comparator>
         <comparator>
           <type>RegexpComparator</type>
           <expected-output>^\s*rather than a number of bytes\.( )*</expected-output>

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitShm.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.hdfs.ExtendedBlockId;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
 import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
@@ -38,8 +39,7 @@ public class TestShortCircuitShm {
   public static final Logger LOG = LoggerFactory.getLogger(
       TestShortCircuitShm.class);
   
-  private static final File TEST_BASE =
-      new File(System.getProperty("test.build.data", "/tmp"));
+  private static final File TEST_BASE = GenericTestUtils.getTestDir();
 
   @Before
   public void before() {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.TestJettyHelper;
 import org.junit.AfterClass;
 import org.junit.runner.RunWith;
@@ -37,8 +38,8 @@ import java.util.UUID;
 public class TestHttpFSFWithSWebhdfsFileSystem
   extends TestHttpFSWithHttpFSFileSystem {
   private static String classpathDir;
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + UUID.randomUUID();
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(UUID.randomUUID().toString());
 
   private static Configuration sslConf;
 

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.TestDirHelper;
 import org.junit.Assert;
 import org.junit.runner.RunWith;
@@ -41,9 +42,7 @@ public class TestHttpFSFileSystemLocalFileSystem extends BaseTestHttpFSWith {
 
   static {
     new TestDirHelper();
-    String prefix =
-      System.getProperty("test.build.dir", "target/test-dir") + "/local";
-    File file = new File(prefix);
+    File file = GenericTestUtils.getTestDir("local");
     file.mkdirs();
     PATH_PREFIX = file.getAbsolutePath();
   }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3HttpServer.java

@@ -30,13 +30,14 @@ import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestNfs3HttpServer {
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestNfs3HttpServer.class.getSimpleName();
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(TestNfs3HttpServer.class.getSimpleName());
   private static NfsConfiguration conf = new NfsConfiguration();
   private static MiniDFSCluster cluster;
   private static String keystoresDir;

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

@@ -60,7 +60,7 @@ public class BlockPlacementStatusWithUpgradeDomain implements
 
   private boolean isUpgradeDomainPolicySatisfied() {
     if (numberOfReplicas <= upgradeDomainFactor) {
-      return (numberOfReplicas == upgradeDomains.size());
+      return (numberOfReplicas <= upgradeDomains.size());
     } else {
       return upgradeDomains.size() >= upgradeDomainFactor;
     }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -502,6 +502,14 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
         CachedBlock cblock = it.next();
         BlockInfo blockInfo = blockManager.
             getStoredBlock(new Block(cblock.getBlockId()));
+        if (blockInfo == null) {
+          // Cannot find this block on the NameNode, skip this block from
+          // capacity calculation. Later logic will handle this block.
+          LOG.debug("Block {}: cannot be found in block manager and hence"
+              + " skipped from calculation for node {}.", cblock.getBlockId(),
+              dn.getDatanodeUuid());
+          continue;
+        }
         if (blockInfo.getNumBytes() > remaining) {
           LOG.debug("Block {}: removing from PENDING_CACHED for node {} "
                   + "because it cannot fit in remaining cache size {}.",

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

@@ -416,7 +416,7 @@ public class EncryptionZoneManager {
     int index = 0;
     for (Map.Entry<Long, EncryptionZoneInt> entry : encryptionZones
         .entrySet()) {
-      ret[index] = entry.getValue().getKeyName();
+      ret[index++] = entry.getValue().getKeyName();
     }
     return ret;
   }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java

@@ -370,6 +370,9 @@ final class FSDirEncryptionZoneOp {
           } else {
             NameNode.LOG.debug("Failed to warm up EDEKs.", ioe);
           }
+        } catch (Exception e) {
+          NameNode.LOG.error("Cannot warm up EDEKs.", e);
+          throw e;
         }
         try {
           Thread.sleep(retryInterval);

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -77,11 +77,11 @@ public final class FSImageFormatPBINode {
   private final static int USER_STRID_OFFSET = 40;
   private final static int GROUP_STRID_OFFSET = 16;
 
-  private static final int ACL_ENTRY_NAME_MASK = (1 << 24) - 1;
-  private static final int ACL_ENTRY_NAME_OFFSET = 6;
-  private static final int ACL_ENTRY_TYPE_OFFSET = 3;
-  private static final int ACL_ENTRY_SCOPE_OFFSET = 5;
-  private static final int ACL_ENTRY_PERM_MASK = 7;
+  public static final int ACL_ENTRY_NAME_MASK = (1 << 24) - 1;
+  public static final int ACL_ENTRY_NAME_OFFSET = 6;
+  public static final int ACL_ENTRY_TYPE_OFFSET = 3;
+  public static final int ACL_ENTRY_SCOPE_OFFSET = 5;
+  public static final int ACL_ENTRY_PERM_MASK = 7;
   private static final int ACL_ENTRY_TYPE_MASK = 3;
   private static final int ACL_ENTRY_SCOPE_MASK = 1;
   private static final FsAction[] FSACTION_VALUES = FsAction.values();

+ 18 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -118,6 +118,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED";
   public static final String NONEXISTENT_STATUS = "does not exist";
   public static final String FAILURE_STATUS = "FAILED";
+  public static final String UNDEFINED = "undefined";
 
   private final NameNode namenode;
   private final BlockManager blockManager;
@@ -141,6 +142,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private boolean showCorruptFileBlocks = false;
 
   private boolean showReplicaDetails = false;
+  private boolean showUpgradeDomains = false;
   private long staleInterval;
   private Tracer tracer;
 
@@ -222,11 +224,15 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       else if (key.equals("racks")) { this.showRacks = true; }
       else if (key.equals("replicadetails")) {
         this.showReplicaDetails = true;
-      }
-      else if (key.equals("storagepolicies")) { this.showStoragePolcies = true; }
-      else if (key.equals("showprogress")) { this.showprogress = true; }
-      else if (key.equals("openforwrite")) {this.showOpenFiles = true; }
-      else if (key.equals("listcorruptfileblocks")) {
+      } else if (key.equals("upgradedomains")) {
+        this.showUpgradeDomains = true;
+      } else if (key.equals("storagepolicies")) {
+        this.showStoragePolcies = true;
+      } else if (key.equals("showprogress")) {
+        this.showprogress = true;
+      } else if (key.equals("openforwrite")) {
+        this.showOpenFiles = true;
+      } else if (key.equals("listcorruptfileblocks")) {
         this.showCorruptFileBlocks = true;
       } else if (key.equals("startblockafter")) {
         this.currentCookie[0] = pmap.get("startblockafter")[0];
@@ -550,7 +556,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    * For striped block group, display info of each internal block.
    */
   private String getReplicaInfo(BlockInfo storedBlock) {
-    if (!(showLocations || showRacks || showReplicaDetails)) {
+    if (!(showLocations || showRacks || showReplicaDetails ||
+        showUpgradeDomains)) {
       return "";
     }
     final boolean isComplete = storedBlock.isComplete();
@@ -568,6 +575,11 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         sb.append(new DatanodeInfoWithStorage(dnDesc, storage.getStorageID(),
             storage.getStorageType()));
       }
+      if (showUpgradeDomains) {
+        String upgradeDomain = (dnDesc.getUpgradeDomain() != null) ?
+            dnDesc.getUpgradeDomain() : UNDEFINED;
+        sb.append("(ud=" + upgradeDomain +")");
+      }
       if (showReplicaDetails) {
         Collection<DatanodeDescriptor> corruptReplicas =
             blockManager.getCorruptReplicas(storedBlock);

+ 11 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java

@@ -77,7 +77,8 @@ public class DFSck extends Configured implements Tool {
   private static final String USAGE = "Usage: hdfs fsck <path> "
       + "[-list-corruptfileblocks | "
       + "[-move | -delete | -openforwrite] "
-      + "[-files [-blocks [-locations | -racks | -replicaDetails]]]] "
+      + "[-files [-blocks [-locations | -racks | -replicaDetails | " +
+          "-upgradedomains]]]] "
       + "[-includeSnapshots] [-showprogress] "
       + "[-storagepolicies] [-blockId <blk_Id>]\n"
       + "\t<path>\tstart checking from this path\n"
@@ -95,6 +96,8 @@ public class DFSck extends Configured implements Tool {
       + "\t-files -blocks -racks" 
       + "\tprint out network topology for data-node locations\n"
       + "\t-files -blocks -replicaDetails\tprint out each replica details \n"
+      + "\t-files -blocks -upgradedomains\tprint out upgrade domains for " +
+          "every block\n"
       + "\t-storagepolicies\tprint out storage policy summary for the blocks\n"
       + "\t-showprogress\tshow progress in output. Default is OFF (no progress)\n"
       + "\t-blockId\tprint out which file this blockId belongs to, locations"
@@ -272,10 +275,13 @@ public class DFSck extends Configured implements Tool {
       else if (args[idx].equals("-racks")) { url.append("&racks=1"); }
       else if (args[idx].equals("-replicaDetails")) {
         url.append("&replicadetails=1");
-      }
-      else if (args[idx].equals("-storagepolicies")) { url.append("&storagepolicies=1"); }
-      else if (args[idx].equals("-showprogress")) { url.append("&showprogress=1"); }
-      else if (args[idx].equals("-list-corruptfileblocks")) {
+      } else if (args[idx].equals("-upgradedomains")) {
+        url.append("&upgradedomains=1");
+      } else if (args[idx].equals("-storagepolicies")) {
+        url.append("&storagepolicies=1");
+      } else if (args[idx].equals("-showprogress")) {
+        url.append("&showprogress=1");
+      } else if (args[idx].equals("-list-corruptfileblocks")) {
         url.append("&listcorruptfileblocks=1");
         doListCorruptFileBlocks = true;
       } else if (args[idx].equals("-includeSnapshots")) {

+ 27 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java

@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_NAME_MASK;
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_NAME_OFFSET;
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_SCOPE_OFFSET;
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.ACL_ENTRY_TYPE_OFFSET;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_MASK;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAME_OFFSET;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.XATTR_NAMESPACE_OFFSET;
@@ -49,10 +53,12 @@ import com.google.common.io.CountingOutputStream;
 import com.google.common.primitives.Ints;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.TextFormat;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
@@ -66,6 +72,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection.DiffEntry;
@@ -131,6 +138,8 @@ class OfflineImageReconstructor {
    */
   private int latestStringId = 0;
 
+  private static final String EMPTY_STRING = "";
+
   private OfflineImageReconstructor(CountingOutputStream out,
       InputStreamReader reader) throws XMLStreamException {
     this.out = out;
@@ -731,10 +740,25 @@ class OfflineImageReconstructor {
     // Will check remaining keys and serialize in processINodeXml
   }
 
-  private INodeSection.AclFeatureProto.Builder aclXmlToProto(Node acl)
+  private INodeSection.AclFeatureProto.Builder aclXmlToProto(Node acls)
       throws IOException {
-    // TODO: support ACLs
-    throw new IOException("ACLs are not supported yet.");
+    AclFeatureProto.Builder b = AclFeatureProto.newBuilder();
+    while (true) {
+      Node acl = acls.removeChild(INODE_SECTION_ACL);
+      if (acl == null) {
+        break;
+      }
+      String val = acl.getVal();
+      AclEntry entry = AclEntry.parseAclEntry(val, true);
+      int nameId = registerStringId(entry.getName() == null ? EMPTY_STRING
+          : entry.getName());
+      int v = ((nameId & ACL_ENTRY_NAME_MASK) << ACL_ENTRY_NAME_OFFSET)
+          | (entry.getType().ordinal() << ACL_ENTRY_TYPE_OFFSET)
+          | (entry.getScope().ordinal() << ACL_ENTRY_SCOPE_OFFSET)
+          | (entry.getPermission().ordinal());
+      b.addEntries(v);
+    }
+    return b;
   }
 
   private INodeSection.XAttrFeatureProto.Builder xattrsXmlToProto(Node xattrs)

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -179,6 +179,13 @@
           data-target="#modal-upload-file" title="Upload Files">
             <span class="glyphicon glyphicon-cloud-upload"></span>
         </button>
+        <button class="btn btn-default dropdown-toggle" type="button"
+          data-toggle="dropdown" title="Cut & Paste">
+        <span class="glyphicon glyphicon-list-alt"></span></button>
+        <ul class="dropdown-menu cut-paste">
+          <li><a id="explorer-cut">Cut</a></li>
+          <li><a id="explorer-paste">Paste</a></li>
+        </ul>
       </div>
     </div>
 
@@ -236,6 +243,7 @@
       <table class="table" id="table-explorer">
         <thead>
           <tr>
+            <th><input type="checkbox" id="file-selector-all"></th>
             <th title="Permissions">Permission</th>
             <th>Owner</th>
             <th>Group</th>
@@ -251,6 +259,7 @@
           {#FileStatus}
           <tr inode-path="{pathSuffix}" data-permission="{permission}"
             class="explorer-entry">
+            <td><input type="checkbox" class="file_selector"> </td>
             <td><span class="explorer-perm-links editable-click">
               {type|helper_to_directory}{permission|helper_to_permission}
               {aclBit|helper_to_acl_bit}

+ 67 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -310,22 +310,28 @@
           var absolute_file_path = append_path(current_directory, inode_name);
           delete_path(inode_name, absolute_file_path);
         });
-          
-          $('#table-explorer').dataTable( {
-              'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
-              'columns': [
-                  {'searchable': false }, //Permissions
-                  null, //Owner
-                  null, //Group
-                  { 'searchable': false, 'render': func_size_render}, //Size
-                  { 'searchable': false, 'render': func_time_render}, //Last Modified
-                  { 'searchable': false }, //Replication
-                  null, //Block Size
-                  null, //Name
-                  { 'sortable' : false } //Trash
-              ],
-              "deferRender": true
-          });
+
+        $('#file-selector-all').click(function() {
+          $('.file_selector').prop('checked', $('#file-selector-all')[0].checked );
+        });
+
+        //This needs to be last because it repaints the table
+        $('#table-explorer').dataTable( {
+          'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
+          'columns': [
+            { 'orderable' : false }, //select
+            {'searchable': false }, //Permissions
+            null, //Owner
+            null, //Group
+            { 'searchable': false, 'render': func_size_render}, //Size
+            { 'searchable': false, 'render': func_time_render}, //Last Modified
+            { 'searchable': false }, //Replication
+            null, //Block Size
+            null, //Name
+            { 'orderable' : false } //Trash
+          ],
+          "deferRender": true
+        });
       });
     }).error(network_error_handler(url));
   }
@@ -417,5 +423,50 @@
     }
   });
 
+  //Store the list of files which have been checked into session storage
+  function store_selected_files(current_directory) {
+    sessionStorage.setItem("source_directory", current_directory);
+    var selected_files = $("input:checked.file_selector");
+    var selected_file_names = new Array();
+    selected_files.each(function(index) {
+      selected_file_names[index] = $(this).closest('tr').attr('inode-path');
+    })
+    sessionStorage.setItem("selected_file_names", JSON.stringify(selected_file_names));
+    alert("Cut " + selected_file_names.length + " files/directories");
+  }
+
+  //Retrieve the list of files from session storage and rename them to the current
+  //directory
+  function paste_selected_files() {
+    var files = JSON.parse(sessionStorage.getItem("selected_file_names"));
+    var source_directory = sessionStorage.getItem("source_directory");
+    $.each(files, function(index, value) {
+      var url = "/webhdfs/v1"
+        + encode_path(append_path(source_directory, value))
+        + '?op=RENAME&destination='
+        + encode_path(append_path(current_directory, value));
+      $.ajax({
+        type: 'PUT',
+        url: url
+      }).done(function(data) {
+        if(index == files.length - 1) {
+          browse_directory(current_directory);
+        }
+      }).error(function(jqXHR, textStatus, errorThrown) {
+        show_err_msg("Couldn't move file " + value + ". " + errorThrown);
+      });
+
+    })
+  }
+
+  $('#explorer-cut').click(function() {
+    store_selected_files(current_directory);
+  });
+
+  $('#explorer-paste').click(function() {
+    paste_selected_files();
+  });
+
+
   init();
 })();

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css

@@ -285,4 +285,11 @@ header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
     margin-bottom: 0;
     font-weight: normal;
     cursor: pointer;
+}
+
+.cut-paste {
+  width: 75px;
+  min-width: 75px;
+  float: right;
+  left: 75px;
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -117,7 +117,7 @@ Usage:
        hdfs fsck <path>
               [-list-corruptfileblocks |
               [-move | -delete | -openforwrite]
-              [-files [-blocks [-locations | -racks | -replicaDetails]]]
+              [-files [-blocks [-locations | -racks | -replicaDetails | -upgradedomains]]]
               [-includeSnapshots] [-showprogress]
               [-storagepolicies] [-blockId <blk_Id>]
 
@@ -130,6 +130,7 @@ Usage:
 | `-files` `-blocks` `-locations` | Print out locations for every block. |
 | `-files` `-blocks` `-racks` | Print out network topology for data-node locations. |
 | `-files` `-blocks` `-replicaDetails` | Print out each replica details. |
+| `-files` `-blocks` `-upgradedomains` | Print out upgrade domains for every block. |
 | `-includeSnapshots` | Include snapshot data if the given path indicates a snapshottable directory or there are snapshottable directories under it. |
 | `-list-corruptfileblocks` | Print out list of missing blocks and files they belong to. |
 | `-move` | Move corrupted files to /lost+found. |

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsRollingUpgrade.md

@@ -172,7 +172,7 @@ Then, the files created before or after *T* remain available in HDFS.
 The files deleted before or after *T* remain deleted in HDFS.
 
 A newer release is downgradable to the pre-upgrade release
-only if both the namenode layout version and the datenode layout version
+only if both the namenode layout version and the datanode layout version
 are not changed between these two releases.
 
 In a HA cluster,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.tools.CryptoAdmin;
 import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,8 +64,7 @@ public class TestCryptoAdminCLI extends CLITestHelperDFS {
         HDFSPolicyProvider.class, PolicyProvider.class);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
 
-    tmpDir = new File(System.getProperty("test.build.data", "target"),
-        UUID.randomUUID().toString()).getAbsoluteFile();
+    tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
     final Path jksPath = new Path(tmpDir.toString(), "test.jks");
     conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());

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

@@ -557,27 +557,25 @@ public class TestEnhancedByteBufferAccess {
    */
   @Test
   public void testIndirectFallbackReads() throws Exception {
-    final File TEST_DIR = new File(
-      System.getProperty("test.build.data","build/test/data"));
-    final String TEST_PATH = TEST_DIR + File.separator +
-        "indirectFallbackTestFile";
+    final String testPath = GenericTestUtils
+        .getTestDir("indirectFallbackTestFile").getAbsolutePath();
     final int TEST_FILE_LENGTH = 16385;
     final int RANDOM_SEED = 23453;
     FileOutputStream fos = null;
     FileInputStream fis = null;
     try {
-      fos = new FileOutputStream(TEST_PATH);
+      fos = new FileOutputStream(testPath);
       Random random = new Random(RANDOM_SEED);
       byte original[] = new byte[TEST_FILE_LENGTH];
       random.nextBytes(original);
       fos.write(original);
       fos.close();
       fos = null;
-      fis = new FileInputStream(TEST_PATH);
+      fis = new FileInputStream(testPath);
       testFallbackImpl(fis, original);
     } finally {
       IOUtils.cleanup(LOG, fos, fis);
-      new File(TEST_PATH).delete();
+      new File(testPath).delete();
     }
   }
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -50,9 +51,8 @@ public class TestSWebHdfsFileContextMainOperations
 
   private static final HdfsConfiguration CONF = new HdfsConfiguration();
 
-  private static final String BASEDIR =
-      System.getProperty("test.build.dir", "target/test-dir") + "/"
-          + TestSWebHdfsFileContextMainOperations.class.getSimpleName();
+  private static final String BASEDIR = GenericTestUtils
+      .getTempPath(TestSWebHdfsFileContextMainOperations.class.getSimpleName());
   protected static int numBlocks = 2;
   protected static final byte[] data = getFileData(numBlocks,
       getDefaultBlockSize());

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -63,7 +64,7 @@ public class TestWebHdfsFileContextMainOperations
 
   @Override
   protected FileContextTestHelper createFileContextHelper() {
-    return new FileContextTestHelper("/tmp/TestWebHdfsFileContextMainOperations");
+    return new FileContextTestHelper();
   }
 
   public URI getWebhdfsUrl() {
@@ -88,8 +89,8 @@ public class TestWebHdfsFileContextMainOperations
   public void setUp() throws Exception {
     URI webhdfsUrlReal = getWebhdfsUrl();
     Path testBuildData = new Path(
-        webhdfsUrlReal + "/build/test/data/" + RandomStringUtils
-            .randomAlphanumeric(10));
+        webhdfsUrlReal + "/" + GenericTestUtils.DEFAULT_TEST_DATA_PATH
+            + RandomStringUtils.randomAlphanumeric(10));
     Path rootPath = new Path(testBuildData, "root-uri");
 
     localFsRootPath = rootPath.makeQualified(webhdfsUrlReal, null);

+ 8 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -103,7 +103,6 @@ import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResour
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -142,7 +141,8 @@ public class MiniDFSCluster {
   private static final String NAMESERVICE_ID_PREFIX = "nameserviceId";
   private static final Log LOG = LogFactory.getLog(MiniDFSCluster.class);
   /** System property to set the data dir: {@value} */
-  public static final String PROP_TEST_BUILD_DATA = "test.build.data";
+  public static final String PROP_TEST_BUILD_DATA =
+      GenericTestUtils.SYSPROP_TEST_DATA_DIR;
   /** Configuration option to set the data dir: {@value} */
   public static final String HDFS_MINIDFS_BASEDIR = "hdfs.minidfs.basedir";
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
@@ -1928,12 +1928,11 @@ public class MiniDFSCluster {
     ShutdownHookManager.get().clearShutdownHooks();
     if (base_dir != null) {
       if (deleteDfsDir) {
-        base_dir.delete();
+        FileUtil.fullyDelete(base_dir);
       } else {
-        base_dir.deleteOnExit();
+        FileUtil.fullyDeleteOnExit(base_dir);
       }
     }
-
   }
   
   /**
@@ -2735,13 +2734,13 @@ public class MiniDFSCluster {
 
   /**
    * Get the base directory for any DFS cluster whose configuration does
-   * not explicitly set it. This is done by retrieving the system property
-   * {@link #PROP_TEST_BUILD_DATA} (defaulting to "build/test/data" ),
-   * and returning that directory with a subdir of /dfs.
+   * not explicitly set it. This is done via
+   * {@link GenericTestUtils#getTestDir()}.
    * @return a directory for use as a miniDFS filesystem.
    */
   public static String getBaseDirectory() {
-    return System.getProperty(PROP_TEST_BUILD_DATA, "build/test/data") + "/dfs/";
+    return GenericTestUtils.getTestDir("dfs").getAbsolutePath()
+        + File.separator;
   }
 
   /**

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java

@@ -115,9 +115,7 @@ public class TestAppendSnapshotTruncate {
     dfs.mkdirs(dir);
     dfs.allowSnapshot(dir);
 
-    final File localDir = new File(
-        System.getProperty("test.build.data", "target/test/data")
-        + dirPathString);
+    final File localDir = GenericTestUtils.getTestDir(dirPathString);
     if (localDir.exists()) {
       FileUtil.fullyDelete(localDir);
     }

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

@@ -3274,8 +3274,8 @@ public class TestDFSShell {
       assertTrue(e.getMessage().contains("Invalid path name /.reserved"));
     }
 
-    final String testdir = System.getProperty("test.build.data")
-        + "/TestDFSShell-testCopyReserved";
+    final String testdir = GenericTestUtils.getTempPath(
+        "TestDFSShell-testCopyReserved");
     final Path hdfsTestDir = new Path(testdir);
     writeFile(fs, new Path(testdir, "testFileForPut"));
     final Path src = new Path(hdfsTestDir, "srcfile");

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -89,8 +89,10 @@ public class TestDFSUpgradeFromImage {
   static {
     upgradeConf = new HdfsConfiguration();
     upgradeConf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // block scanning off
-    if (System.getProperty("test.build.data") == null) { // to allow test to be run outside of Maven
-      System.setProperty("test.build.data", "build/test/data");
+    if (System.getProperty(GenericTestUtils.SYSPROP_TEST_DATA_DIR) == null) {
+      // to allow test to be run outside of Maven
+      System.setProperty(GenericTestUtils.SYSPROP_TEST_DATA_DIR,
+          GenericTestUtils.DEFAULT_TEST_DATA_DIR);
     }
   }
   
@@ -105,19 +107,19 @@ public class TestDFSUpgradeFromImage {
   
   void unpackStorage(String tarFileName, String referenceName)
       throws IOException {
-    String tarFile = System.getProperty("test.cache.data", "build/test/cache")
+    String tarFile = System.getProperty("test.cache.data", "target/test/cache")
         + "/" + tarFileName;
-    String dataDir = System.getProperty("test.build.data", "build/test/data");
+    File dataDir = GenericTestUtils.getTestDir();
     File dfsDir = new File(dataDir, "dfs");
     if ( dfsDir.exists() && !FileUtil.fullyDelete(dfsDir) ) {
       throw new IOException("Could not delete dfs directory '" + dfsDir + "'");
     }
     LOG.info("Unpacking " + tarFile);
-    FileUtil.unTar(new File(tarFile), new File(dataDir));
+    FileUtil.unTar(new File(tarFile), dataDir);
     //Now read the reference info
     
     BufferedReader reader = new BufferedReader(new FileReader(
-        System.getProperty("test.cache.data", "build/test/cache")
+        System.getProperty("test.cache.data", "target/test/cache")
             + "/" + referenceName));
     String line;
     while ( (line = reader.readLine()) != null ) {
@@ -631,10 +633,10 @@ public class TestDFSUpgradeFromImage {
     unpackStorage(HADOOP1_BBW_IMAGE, HADOOP_DFS_DIR_TXT);
     Configuration conf = new Configuration(upgradeConf);
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, 
-        System.getProperty("test.build.data") + File.separator + 
+        GenericTestUtils.getTempPath(
         "dfs" + File.separator + 
         "data" + File.separator + 
-        "data1");
+        "data1"));
     upgradeAndVerify(new MiniDFSCluster.Builder(conf).
           numDataNodes(1).enableManagedDfsDirsRedundancy(false).
           manageDataDfsDirs(false), null);

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

@@ -874,8 +874,7 @@ public class TestDFSUtil {
 
   @Test
   public void testGetPassword() throws Exception {
-    File testDir = new File(System.getProperty("test.build.data",
-        "target/test-dir"));
+    File testDir = GenericTestUtils.getTestDir();
 
     Configuration conf = new Configuration();
     final Path jksPath = new Path(testDir.toString(), "test.jks");

+ 11 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 import java.io.File;
@@ -42,11 +43,11 @@ public class TestDatanodeLayoutUpgrade {
     TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
     upgrade.unpackStorage(HADOOP24_DATANODE, HADOOP_DATANODE_DIR_TXT);
     Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-        new File(System.getProperty("test.build.data"),
+    conf.set(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, GenericTestUtils.getTestDir(
             "dfs" + File.separator + "data").toURI().toString());
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-        new File(System.getProperty("test.build.data"),
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, GenericTestUtils.getTestDir(
             "dfs" + File.separator + "name").toURI().toString());
     upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
     .manageDataDfsDirs(false).manageNameDfsDirs(false), null);
@@ -61,12 +62,12 @@ public class TestDatanodeLayoutUpgrade {
     TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
     upgrade.unpackStorage(HADOOP_56_DN_LAYOUT, HADOOP_56_DN_LAYOUT_TXT);
     Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-        new File(System.getProperty("test.build.data"), "dfs" + File.separator
-            + "data").toURI().toString());
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-        new File(System.getProperty("test.build.data"), "dfs" + File.separator
-            + "name").toURI().toString());
+    conf.set(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, GenericTestUtils.getTestDir(
+            "dfs" + File.separator + "data").toURI().toString());
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, GenericTestUtils.getTestDir(
+            "dfs" + File.separator + "name").toURI().toString());
     upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .manageDataDfsDirs(false).manageNameDfsDirs(false), null);
   }

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java

@@ -87,12 +87,12 @@ public class TestDatanodeStartupFixesLegacyStorageIDs {
 
   private static void initStorageDirs(final Configuration conf,
                                       final String testName) {
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
-             System.getProperty("test.build.data") + File.separator +
-                 testName + File.separator + "dfs" + File.separator + "data");
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-             System.getProperty("test.build.data") + File.separator +
-                 testName + File.separator + "dfs" + File.separator + "name");
+    conf.set(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, GenericTestUtils.getTempPath(
+            testName + File.separator + "dfs" + File.separator + "data"));
+    conf.set(
+        DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, GenericTestUtils.getTempPath(
+            testName + File.separator + "dfs" + File.separator + "name"));
 
   }
 

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java

@@ -100,9 +100,14 @@ public class TestEncryptionZonesWithKMS extends TestEncryptionZones {
 
   @Test(timeout = 120000)
   public void testWarmupEDEKCacheOnStartup() throws Exception {
-    final Path zonePath = new Path("/TestEncryptionZone");
+    Path zonePath = new Path("/TestEncryptionZone");
     fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false);
     dfsAdmin.createEncryptionZone(zonePath, TEST_KEY, NO_TRASH);
+    final String anotherKey = "k2";
+    zonePath = new Path("/TestEncryptionZone2");
+    DFSTestUtil.createKey(anotherKey, cluster, conf);
+    fsWrapper.mkdir(zonePath, FsPermission.getDirDefault(), false);
+    dfsAdmin.createEncryptionZone(zonePath, anotherKey, NO_TRASH);
 
     @SuppressWarnings("unchecked")
     KMSClientProvider spy = (KMSClientProvider) Whitebox

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java

@@ -32,13 +32,14 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Test;
 
 public class TestFetchImage {
   
-  private static final File FETCHED_IMAGE_FILE = new File(
-      System.getProperty("test.build.dir"), "target/fetched-image-dir");
+  private static final File FETCHED_IMAGE_FILE =
+      GenericTestUtils.getTestDir("target/fetched-image-dir");
   // Shamelessly stolen from NNStorage.
   private static final Pattern IMAGE_REGEX = Pattern.compile("fsimage_(\\d+)");
 

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

@@ -482,7 +482,7 @@ public class TestPread {
     Configuration conf = new HdfsConfiguration();
     FileSystem fileSys = FileSystem.getLocal(conf);
     try {
-      Path file1 = new Path("build/test/data", "preadtest.dat");
+      Path file1 = new Path(GenericTestUtils.getTempPath("preadtest.dat"));
       writeFile(fileSys, file1);
       pReadFile(fileSys, file1);
       cleanupFile(fileSys, file1);

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

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 /**
@@ -204,7 +205,7 @@ public class TestSeekBug {
     Configuration conf = new HdfsConfiguration();
     FileSystem fileSys = FileSystem.getLocal(conf);
     try {
-      Path file1 = new Path("build/test/data", "seektest.dat");
+      Path file1 = new Path(GenericTestUtils.getTempPath("seektest.dat"));
       DFSTestUtil.createFile(fileSys, file1, ONEMB, ONEMB,
           fileSys.getDefaultBlockSize(file1),
           fileSys.getDefaultReplication(file1), seed);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -74,8 +75,8 @@ public abstract class SaslDataTransferTestCase {
 
   @BeforeClass
   public static void initKdc() throws Exception {
-    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
-      SaslDataTransferTestCase.class.getSimpleName());
+    baseDir = GenericTestUtils
+        .getTestDir(SaslDataTransferTestCase.class.getSimpleName());
     FileUtil.fullyDelete(baseDir);
     assertTrue(baseDir.mkdirs());
 

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

@@ -55,6 +55,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -82,8 +83,8 @@ public class TestSecureNNWithQJM {
 
   @BeforeClass
   public static void init() throws Exception {
-    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
-      TestSecureNNWithQJM.class.getSimpleName());
+    baseDir =
+        GenericTestUtils.getTestDir(TestSecureNNWithQJM.class.getSimpleName());
     FileUtil.fullyDelete(baseDir);
     assertTrue(baseDir.mkdirs());
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -211,8 +211,7 @@ public class TestBalancer {
   }
 
   static void initSecureConf(Configuration conf) throws Exception {
-    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
-        TestBalancer.class.getSimpleName());
+    baseDir = GenericTestUtils.getTestDir(TestBalancer.class.getSimpleName());
     FileUtil.fullyDelete(baseDir);
     assertTrue(baseDir.mkdirs());
 

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementStatusWithUpgradeDomain.java

@@ -0,0 +1,83 @@
+/**
+ * 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.server.blockmanagement;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit tests for BlockPlacementStatusWithUpgradeDomain class.
+ */
+public class TestBlockPlacementStatusWithUpgradeDomain {
+
+  private Set<String> upgradeDomains;
+  private BlockPlacementStatusDefault bpsd =
+      mock(BlockPlacementStatusDefault.class);
+
+  @Before
+  public void setup() {
+    upgradeDomains = new HashSet<String>();
+    upgradeDomains.add("1");
+    upgradeDomains.add("2");
+    upgradeDomains.add("3");
+    when(bpsd.isPlacementPolicySatisfied()).thenReturn(true);
+  }
+
+  @Test
+  public void testIsPolicySatisfiedParentFalse() {
+    when(bpsd.isPlacementPolicySatisfied()).thenReturn(false);
+    BlockPlacementStatusWithUpgradeDomain bps =
+        new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3);
+
+    // Parent policy is not satisfied but upgrade domain policy is
+    assertFalse(bps.isPlacementPolicySatisfied());
+  }
+
+  @Test
+  public void testIsPolicySatisfiedAllEqual() {
+    BlockPlacementStatusWithUpgradeDomain bps =
+        new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 3, 3);
+    // Number of domains, replicas and upgradeDomainFactor is equal and parent
+    // policy is satisfied
+    assertTrue(bps.isPlacementPolicySatisfied());
+  }
+
+  @Test
+  public void testIsPolicySatisifedSmallDomains() {
+    // Number of domains is less than replicas but equal to factor
+    BlockPlacementStatusWithUpgradeDomain bps =
+        new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 3);
+    assertTrue(bps.isPlacementPolicySatisfied());
+
+    // Same as above but replicas is greater than factor
+    bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 2);
+    assertTrue(bps.isPlacementPolicySatisfied());
+
+    // Number of domains is less than replicas and factor
+    bps = new BlockPlacementStatusWithUpgradeDomain(bpsd, upgradeDomains, 4, 4);
+    assertFalse(bps.isPlacementPolicySatisfied());
+  }
+}

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 import java.io.File;
@@ -69,7 +70,7 @@ public class TestDataNodeUUID {
 
   @Test(timeout = 10000)
   public void testUUIDRegeneration() throws Exception {
-    File baseDir = new File(System.getProperty("test.build.data"));
+    File baseDir = GenericTestUtils.getTestDir();
     File disk1 = new File(baseDir, "disk1");
     File disk2 = new File(baseDir, "disk2");
 

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

@@ -47,8 +47,7 @@ public class TestDataStorage {
   private final static String BUILD_VERSION = "2.0";
   private final static String SOFTWARE_VERSION = "2.0";
   private final static long CTIME = 1;
-  private final static File TEST_DIR =
-      new File(System.getProperty("test.build.data") + "/dstest");
+  private final static File TEST_DIR = GenericTestUtils.getTestDir("dstest");
   private final static StartupOption START_OPT = StartupOption.REGULAR;
 
   private DataNode mockDN = Mockito.mock(DataNode.class);

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

@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 
 import java.io.File;
 
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,9 +43,8 @@ public class TestCreateEditsLog {
 
   private static final File HDFS_DIR = new File(
     MiniDFSCluster.getBaseDirectory()).getAbsoluteFile();
-  private static final File TEST_DIR = new File(
-    System.getProperty("test.build.data", "build/test/data"),
-    "TestCreateEditsLog").getAbsoluteFile();
+  private static final File TEST_DIR =
+      GenericTestUtils.getTestDir("TestCreateEditsLog");
 
   private MiniDFSCluster cluster;
 

+ 18 - 47
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java

@@ -25,7 +25,6 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
@@ -55,6 +54,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DecommissionManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.After;
@@ -71,10 +71,8 @@ public class TestDecommissioningStatus {
   private static final int numDatanodes = 2;
   private static MiniDFSCluster cluster;
   private static FileSystem fileSys;
-  private static Path excludeFile;
-  private static FileSystem localFileSys;
+  private static HostsFileWriter hostsFileWriter;
   private static Configuration conf;
-  private static Path dir;
 
   final ArrayList<String> decommissionedNodes = new ArrayList<String>(numDatanodes);
   
@@ -85,14 +83,8 @@ public class TestDecommissioningStatus {
         false);
 
     // Set up the hosts/exclude files.
-    localFileSys = FileSystem.getLocal(conf);
-    Path workingDir = localFileSys.getWorkingDirectory();
-    dir = new Path(workingDir, "build/test/data/work-dir/decommission");
-    assertTrue(localFileSys.mkdirs(dir));
-    excludeFile = new Path(dir, "exclude");
-    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
-    Path includeFile = new Path(dir, "include");
-    conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath());
+    hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(conf, "work-dir/decommission");
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 
         1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
@@ -102,9 +94,6 @@ public class TestDecommissioningStatus {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
     conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY, 1);
 
-    writeConfigFile(localFileSys, excludeFile, null);
-    writeConfigFile(localFileSys, includeFile, null);
-
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
     cluster.waitActive();
     fileSys = cluster.getFileSystem();
@@ -115,31 +104,13 @@ public class TestDecommissioningStatus {
 
   @After
   public void tearDown() throws Exception {
-    if (localFileSys != null ) cleanupFile(localFileSys, dir);
+    if (hostsFileWriter != null) {
+      hostsFileWriter.cleanup();
+    }
     if(fileSys != null) fileSys.close();
     if(cluster != null) cluster.shutdown();
   }
 
-  private static void writeConfigFile(FileSystem fs, Path name,
-      ArrayList<String> nodes) throws IOException {
-
-    // delete if it already exists
-    if (fs.exists(name)) {
-      fs.delete(name, true);
-    }
-
-    FSDataOutputStream stm = fs.create(name);
-
-    if (nodes != null) {
-      for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
-        String node = it.next();
-        stm.writeBytes(node);
-        stm.writeBytes("\n");
-      }
-    }
-    stm.close();
-  }
-
   private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
       short repl) throws IOException {
     // create and write a file that contains three blocks of data
@@ -169,25 +140,25 @@ public class TestDecommissioningStatus {
    * Decommissions the node at the given index
    */
   private String decommissionNode(FSNamesystem namesystem, DFSClient client,
-      FileSystem localFileSys, int nodeIndex) throws IOException {
+      int nodeIndex) throws IOException {
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
 
     String nodename = info[nodeIndex].getXferAddr();
-    decommissionNode(namesystem, localFileSys, nodename);
+    decommissionNode(namesystem, nodename);
     return nodename;
   }
 
   /*
    * Decommissions the node by name
    */
-  private void decommissionNode(FSNamesystem namesystem,
-      FileSystem localFileSys, String dnName) throws IOException {
+  private void decommissionNode(FSNamesystem namesystem, String dnName)
+      throws IOException {
     System.out.println("Decommissioning node: " + dnName);
 
     // write nodename into the exclude file.
     ArrayList<String> nodes = new ArrayList<String>(decommissionedNodes);
     nodes.add(dnName);
-    writeConfigFile(localFileSys, excludeFile, nodes);
+    hostsFileWriter.initExcludeHosts(nodes.toArray(new String[0]));
   }
 
   private void checkDecommissionStatus(DatanodeDescriptor decommNode,
@@ -280,7 +251,7 @@ public class TestDecommissioningStatus {
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     for (int iteration = 0; iteration < numDatanodes; iteration++) {
-      String downnode = decommissionNode(fsn, client, localFileSys, iteration);
+      String downnode = decommissionNode(fsn, client, iteration);
       dm.refreshNodes(conf);
       decommissionedNodes.add(downnode);
       BlockManagerTestUtil.recheckDecommissionState(dm);
@@ -307,7 +278,7 @@ public class TestDecommissioningStatus {
     // Call refreshNodes on FSNamesystem with empty exclude file.
     // This will remove the datanodes from decommissioning list and
     // make them available again.
-    writeConfigFile(localFileSys, excludeFile, null);
+    hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
     st1.close();
     cleanupFile(fileSys, file1);
@@ -337,7 +308,7 @@ public class TestDecommissioningStatus {
     // Decommission the DN.
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
-    decommissionNode(fsn, localFileSys, dnName);
+    decommissionNode(fsn, dnName);
     dm.refreshNodes(conf);
 
     // Stop the DN when decommission is in progress.
@@ -384,7 +355,7 @@ public class TestDecommissioningStatus {
     // Call refreshNodes on FSNamesystem with empty exclude file.
     // This will remove the datanodes from decommissioning list and
     // make them available again.
-    writeConfigFile(localFileSys, excludeFile, null);
+    hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
   }
 
@@ -405,7 +376,7 @@ public class TestDecommissioningStatus {
     FSNamesystem fsn = cluster.getNamesystem();
     final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     DatanodeDescriptor dnDescriptor = dm.getDatanode(dnID);
-    decommissionNode(fsn, localFileSys, dnName);
+    decommissionNode(fsn, dnName);
     dm.refreshNodes(conf);
     BlockManagerTestUtil.recheckDecommissionState(dm);
     assertTrue(dnDescriptor.isDecommissioned());
@@ -416,7 +387,7 @@ public class TestDecommissioningStatus {
 
     // Call refreshNodes on FSNamesystem with empty exclude file to remove the
     // datanode from decommissioning list and make it available again.
-    writeConfigFile(localFileSys, excludeFile, null);
+    hostsFileWriter.initExcludeHost("");
     dm.refreshNodes(conf);
   }
 }

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

@@ -87,8 +87,7 @@ public class TestEditLogFileInputStream {
   @Test(timeout=60000)
   public void testScanCorruptEditLog() throws Exception {
     Configuration conf = new Configuration();
-    File editLog = new File(System.getProperty(
-        "test.build.data", "/tmp"), "testCorruptEditLog");
+    File editLog = new File(GenericTestUtils.getTempPath("testCorruptEditLog"));
 
     LOG.debug("Creating test edit log file: " + editLog);
     EditLogFileOutputStream elos = new EditLogFileOutputStream(conf,

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

@@ -69,8 +69,8 @@ public class TestFSImageWithSnapshot {
 
   private final Path dir = new Path("/TestSnapshot");
   private static final String testDir =
-      System.getProperty("test.build.data", "build/test/data");
-  
+      GenericTestUtils.getTestDir().getAbsolutePath();
+
   Configuration conf;
   MiniDFSCluster cluster;
   FSNamesystem fsn;

+ 78 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -24,9 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -81,6 +79,8 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -90,13 +90,16 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
@@ -116,8 +119,8 @@ import com.google.common.collect.Sets;
  * A JUnit test for doing fsck
  */
 public class TestFsck {
-  static final String auditLogFile = System.getProperty("test.build.dir",
-      "build/test") + "/TestFsck-audit.log";
+  static final String AUDITLOG_FILE =
+      GenericTestUtils.getTempPath("TestFsck-audit.log");
   
   // Pattern for: 
   // allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null
@@ -209,14 +212,15 @@ public class TestFsck {
 
   /** Sets up log4j logger for auditlogs */
   private void setupAuditLogs() throws IOException {
-    File file = new File(auditLogFile);
+    File file = new File(AUDITLOG_FILE);
     if (file.exists()) {
       file.delete();
     }
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
     logger.setLevel(Level.INFO);
     PatternLayout layout = new PatternLayout("%m%n");
-    RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile);
+    RollingFileAppender appender =
+        new RollingFileAppender(layout, AUDITLOG_FILE);
     logger.addAppender(appender);
   }
   
@@ -228,7 +232,7 @@ public class TestFsck {
     BufferedReader reader = null;
     try {
       // Audit log should contain one getfileinfo and one fsck
-      reader = new BufferedReader(new FileReader(auditLogFile));
+      reader = new BufferedReader(new FileReader(AUDITLOG_FILE));
       String line;
 
       // one extra getfileinfo stems from resolving the path
@@ -2008,4 +2012,71 @@ public class TestFsck {
     }
   }
 
+  @Test(timeout = 60000)
+  public void testFsckUpgradeDomain() throws Exception {
+    testUpgradeDomain(false, false);
+    testUpgradeDomain(false, true);
+    testUpgradeDomain(true, false);
+    testUpgradeDomain(true, true);
+  }
+
+  private void testUpgradeDomain(boolean defineUpgradeDomain,
+      boolean displayUpgradeDomain) throws Exception {
+    final short replFactor = 1;
+    final short numDN = 1;
+    final long blockSize = 512;
+    final long fileSize = 1024;
+    final String upgradeDomain = "ud1";
+    final String[] racks = {"/rack1"};
+    final String[] hosts = {"127.0.0.1"};
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replFactor);
+    if (defineUpgradeDomain) {
+      conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY,
+          CombinedHostFileManager.class, HostConfigManager.class);
+      hostsFileWriter.initialize(conf, "temp/fsckupgradedomain");
+    }
+
+    MiniDFSCluster cluster;
+    DistributedFileSystem dfs;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDN).
+        hosts(hosts).racks(racks).build();
+    cluster.waitClusterUp();
+    dfs = cluster.getFileSystem();
+
+    // Configure the upgrade domain on the datanode
+    if (defineUpgradeDomain) {
+      DatanodeAdminProperties dnProp = new DatanodeAdminProperties();
+      DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
+      dnProp.setHostName(datanodeID.getHostName());
+      dnProp.setPort(datanodeID.getXferPort());
+      dnProp.setUpgradeDomain(upgradeDomain);
+      hostsFileWriter.initIncludeHosts(new DatanodeAdminProperties[]{dnProp});
+      cluster.getFileSystem().refreshNodes();
+    }
+
+    // create files
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(dfs, path, fileSize, replFactor, 1000L);
+    DFSTestUtil.waitReplication(dfs, path, replFactor);
+    try {
+      String fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
+          displayUpgradeDomain ? "-upgradedomains" : "-locations");
+      assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
+      String udValue = defineUpgradeDomain ? upgradeDomain :
+          NamenodeFsck.UNDEFINED;
+      assertEquals(displayUpgradeDomain,
+          fsckOut.contains("(ud=" + udValue + ")"));
+    } finally {
+      if (defineUpgradeDomain) {
+        hostsFileWriter.cleanup();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

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

@@ -32,6 +32,7 @@ import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig.Policy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -42,8 +43,8 @@ import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(value = Parameterized.class)
 public class TestNameNodeHttpServer {
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestNameNodeHttpServer.class.getSimpleName();
+  private static final String BASEDIR = GenericTestUtils
+      .getTempPath(TestNameNodeHttpServer.class.getSimpleName());
   private static String keystoresDir;
   private static String sslConfDir;
   private static Configuration conf;

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

@@ -49,6 +49,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.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
@@ -521,8 +522,8 @@ public class TestNameNodeRecovery {
     conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
     conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
       "ns1"), "nn1,nn2");
-    String baseDir = System.getProperty(
-        MiniDFSCluster.PROP_TEST_BUILD_DATA, "build/test/data") + "/dfs/";
+    String baseDir = GenericTestUtils.getTestDir("setupRecoveryTestConf")
+        .getAbsolutePath();
     File nameDir = new File(baseDir, "nameR");
     File secondaryDir = new File(baseDir, "namesecondaryR");
     conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.

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

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 
 import java.io.File;
 import java.io.IOException;
@@ -237,8 +238,8 @@ public class TestNameNodeRespectsBindHostKeys {
     }
   }
 
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestNameNodeRespectsBindHostKeys.class.getSimpleName();
+  private static final String BASEDIR = GenericTestUtils
+      .getTempPath(TestNameNodeRespectsBindHostKeys.class.getSimpleName());
 
   private static void setupSsl() throws Exception {
     Configuration conf = new Configuration();

+ 125 - 36
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestUpgradeDomainBlockPlacementPolicy.java

@@ -17,38 +17,40 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-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.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithUpgradeDomain;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.base.Supplier;
+
 /**
  * End-to-end test case for upgrade domain
  * The test configs upgrade domain for nodes via admin json
@@ -69,15 +71,16 @@ public class TestUpgradeDomainBlockPlacementPolicy {
    *  CombinedHostFileManager won't allow those hosts.
    */
   static final String[] hosts =
-      { "127.0.0.1", "127.0.0.1", "127.0.0.1", "127.0.0.1",
-          "127.0.0.1", "127.0.0.1" };
+      {"127.0.0.1", "127.0.0.1", "127.0.0.1", "127.0.0.1",
+          "127.0.0.1", "127.0.0.1"};
   static final String[] upgradeDomains =
-      { "ud1", "ud2", "ud3", "ud1", "ud2", "ud3" };
+      {"ud5", "ud2", "ud3", "ud1", "ud2", "ud4"};
   static final Set<DatanodeID> expectedDatanodeIDs = new HashSet<>();
   private MiniDFSCluster cluster = null;
   private NamenodeProtocols nameNodeRpc = null;
   private FSNamesystem namesystem = null;
   private PermissionStatus perm = null;
+  private HostsFileWriter hostsFileWriter = new HostsFileWriter();
 
   @Before
   public void setup() throws IOException {
@@ -86,11 +89,10 @@ public class TestUpgradeDomainBlockPlacementPolicy {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE / 2);
     conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
-        BlockPlacementPolicyWithUpgradeDomain.class,
-        BlockPlacementPolicy.class);
+            BlockPlacementPolicyWithUpgradeDomain.class,
+            BlockPlacementPolicy.class);
     conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY,
-        CombinedHostFileManager.class, HostConfigManager.class);
-    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+            CombinedHostFileManager.class, HostConfigManager.class);
     hostsFileWriter.initialize(conf, "temp/upgradedomainpolicy");
 
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(6).racks(racks)
@@ -100,12 +102,12 @@ public class TestUpgradeDomainBlockPlacementPolicy {
     namesystem = cluster.getNamesystem();
     perm = new PermissionStatus("TestDefaultBlockPlacementPolicy", null,
         FsPermission.getDefault());
-    refreshDatanodeAdminProperties(hostsFileWriter);
-    hostsFileWriter.cleanup();
+    refreshDatanodeAdminProperties();
   }
 
   @After
-  public void teardown() {
+  public void teardown() throws IOException {
+    hostsFileWriter.cleanup();
     if (cluster != null) {
       cluster.shutdown();
       cluster = null;
@@ -114,15 +116,18 @@ public class TestUpgradeDomainBlockPlacementPolicy {
 
   /**
    * Define admin properties for these datanodes as follows.
-   * dn0 and dn3 have upgrade domain ud1.
-   * dn1 and dn4 have upgrade domain ud2.
-   * dn2 and dn5 have upgrade domain ud3.
+   * dn0's upgrade domain is ud5.
+   * dn1's upgrade domain is ud2.
+   * dn2's upgrade domain is ud3.
+   * dn3's upgrade domain is ud1.
+   * dn4's upgrade domain is ud2.
+   * dn5's upgrade domain is ud4.
    * dn0 and dn5 are decommissioned.
    * Given dn0, dn1 and dn2 are on rack1 and dn3, dn4 and dn5 are on
    * rack2. Then any block's replicas should be on either
    * {dn1, dn2, d3} or {dn2, dn3, dn4}.
    */
-  private void refreshDatanodeAdminProperties(HostsFileWriter hostsFileWriter)
+  private void refreshDatanodeAdminProperties()
       throws IOException {
     DatanodeAdminProperties[] datanodes = new DatanodeAdminProperties[
         hosts.length];
@@ -138,32 +143,116 @@ public class TestUpgradeDomainBlockPlacementPolicy {
     hostsFileWriter.initIncludeHosts(datanodes);
     cluster.getFileSystem().refreshNodes();
 
+    expectedDatanodeIDs.clear();
     expectedDatanodeIDs.add(cluster.getDataNodes().get(2).getDatanodeId());
     expectedDatanodeIDs.add(cluster.getDataNodes().get(3).getDatanodeId());
   }
 
+  /**
+   * Define admin properties for these datanodes as follows.
+   * dn0's upgrade domain is ud5.
+   * dn1's upgrade domain is ud2.
+   * dn2's upgrade domain is ud3.
+   * dn3's upgrade domain is ud1.
+   * dn4's upgrade domain is ud2.
+   * dn5's upgrade domain is ud4.
+   * dn2 and dn3 are decommissioned.
+   * Given dn0, dn1 and dn2 are on rack1 and dn3, dn4 and dn5 are on
+   * rack2. Then any block's replicas should be on either
+   * {dn0, dn1, d5} or {dn0, dn4, dn5}.
+   */
+  private void refreshDatanodeAdminProperties2()
+      throws IOException {
+    DatanodeAdminProperties[] datanodes = new DatanodeAdminProperties[
+        hosts.length];
+    for (int i = 0; i < hosts.length; i++) {
+      datanodes[i] = new DatanodeAdminProperties();
+      DatanodeID datanodeID = cluster.getDataNodes().get(i).getDatanodeId();
+      datanodes[i].setHostName(datanodeID.getHostName());
+      datanodes[i].setPort(datanodeID.getXferPort());
+      datanodes[i].setUpgradeDomain(upgradeDomains[i]);
+    }
+    datanodes[2].setAdminState(DatanodeInfo.AdminStates.DECOMMISSIONED);
+    datanodes[3].setAdminState(DatanodeInfo.AdminStates.DECOMMISSIONED);
+    hostsFileWriter.initIncludeHosts(datanodes);
+    cluster.getFileSystem().refreshNodes();
+
+    expectedDatanodeIDs.clear();
+    expectedDatanodeIDs.add(cluster.getDataNodes().get(0).getDatanodeId());
+    expectedDatanodeIDs.add(cluster.getDataNodes().get(5).getDatanodeId());
+  }
+
   @Test
   public void testPlacement() throws Exception {
-    String clientMachine = "127.0.0.1";
-    for (int i = 0; i < 5; i++) {
-      String src = "/test-" + i;
-      // Create the file with client machine
-      HdfsFileStatus fileStatus = namesystem.startFile(src, perm,
-          clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true,
-          REPLICATION_FACTOR, DEFAULT_BLOCK_SIZE, null, false);
-      LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine,
-          null, null, fileStatus.getFileId(), null, null);
-
-      assertEquals("Block should be allocated sufficient locations",
-          REPLICATION_FACTOR, locatedBlock.getLocations().length);
-      Set<DatanodeInfo> locs = new HashSet<>(Arrays.asList(
-          locatedBlock.getLocations()));
+    final long fileSize = DEFAULT_BLOCK_SIZE * 5;
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(cluster.getFileSystem(), path, fileSize,
+        REPLICATION_FACTOR, 1000L);
+    LocatedBlocks locatedBlocks =
+        cluster.getFileSystem().getClient().getLocatedBlocks(
+            path.toString(), 0, fileSize);
+    for (LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
+      Set<DatanodeInfo> locs = new HashSet<>();
+      for(DatanodeInfo datanodeInfo : block.getLocations()) {
+        if (datanodeInfo.getAdminState() == DatanodeInfo.AdminStates.NORMAL) {
+          locs.add(datanodeInfo);
+        }
+      }
       for (DatanodeID datanodeID : expectedDatanodeIDs) {
-        locs.contains(datanodeID);
+        assertTrue(locs.contains(datanodeID));
+      }
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testPlacementAfterDecommission() throws Exception {
+    final long fileSize = DEFAULT_BLOCK_SIZE * 5;
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(cluster.getFileSystem(), path, fileSize,
+        REPLICATION_FACTOR, 1000L);
+
+    // Decommission some nodes and wait until decommissions have finished.
+    refreshDatanodeAdminProperties2();
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        boolean successful = true;
+        LocatedBlocks locatedBlocks;
+        try {
+          locatedBlocks =
+              cluster.getFileSystem().getClient().getLocatedBlocks(
+                  path.toString(), 0, fileSize);
+        } catch (IOException ioe) {
+          return false;
+        }
+        for(LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
+          Set<DatanodeInfo> locs = new HashSet<>();
+          for (DatanodeInfo datanodeInfo : block.getLocations()) {
+            if (datanodeInfo.getAdminState() ==
+                DatanodeInfo.AdminStates.NORMAL) {
+              locs.add(datanodeInfo);
+            }
+          }
+          for (DatanodeID datanodeID : expectedDatanodeIDs) {
+            successful = successful && locs.contains(datanodeID);
+          }
+        }
+        return successful;
       }
+    }, 1000, 60000);
 
-      nameNodeRpc.abandonBlock(locatedBlock.getBlock(), fileStatus.getFileId(),
-          src, clientMachine);
+    // Verify block placement policy of each block.
+    LocatedBlocks locatedBlocks;
+    locatedBlocks =
+        cluster.getFileSystem().getClient().getLocatedBlocks(
+            path.toString(), 0, fileSize);
+    for(LocatedBlock block : locatedBlocks.getLocatedBlocks()) {
+      BlockPlacementStatus status = cluster.getNamesystem().getBlockManager().
+          getBlockPlacementPolicy().verifyBlockPlacement(
+              block.getLocations(), REPLICATION_FACTOR);
+      assertTrue(status.isPlacementPolicySatisfied());
     }
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -95,7 +95,7 @@ public class TestRenameWithSnapshots {
   private static FSDirectory fsdir;
   private static DistributedFileSystem hdfs;
   private static final String testDir =
-      System.getProperty("test.build.data", "build/test/data");
+      GenericTestUtils.getTestDir().getAbsolutePath();
   static private final Path dir = new Path("/testRenameWithSnapshots");
   static private final Path sub1 = new Path(dir, "sub1");
   static private final Path file1 = new Path(sub1, "file1");

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

@@ -98,8 +98,8 @@ public class TestSnapshot {
   protected DistributedFileSystem hdfs;
   
   private static final String testDir =
-      System.getProperty("test.build.data", "build/test/data");
-  
+      GenericTestUtils.getTestDir().getAbsolutePath();
+
   @Rule
   public ExpectedException exception = ExpectedException.none();
   

+ 11 - 51
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java

@@ -35,20 +35,18 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.StringTokenizer;
 
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.tools.GetConf.Command;
 import org.apache.hadoop.hdfs.tools.GetConf.CommandHandler;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
@@ -390,42 +388,29 @@ public class TestGetConf {
   public void TestGetConfExcludeCommand() throws Exception{
   	HdfsConfiguration conf = new HdfsConfiguration();
     // 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") + "/Getconf/");
-    Path hostsFile = new Path(dir, "hosts");
-    Path excludeFile = new Path(dir, "exclude");
-    
-    // Setup conf
-    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
-    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
-    writeConfigFile(hostsFile, null);
-    writeConfigFile(excludeFile, null);    
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(conf, "GetConf");
+    Path excludeFile = hostsFileWriter.getExcludeFile();
+
     String[] args = {"-excludeFile"};
     String ret = runTool(conf, args, true);
     assertEquals(excludeFile.toUri().getPath(),ret.trim());
-    cleanupFile(localFileSys, excludeFile.getParent());
+    hostsFileWriter.cleanup();
   }
   
   @Test
   public void TestGetConfIncludeCommand() throws Exception{
   	HdfsConfiguration conf = new HdfsConfiguration();
     // 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") + "/Getconf/");
-    Path hostsFile = new Path(dir, "hosts");
-    Path excludeFile = new Path(dir, "exclude");
-    
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(conf, "GetConf");
+    Path hostsFile = hostsFileWriter.getIncludeFile();
+
     // Setup conf
-    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
-    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
-    writeConfigFile(hostsFile, null);
-    writeConfigFile(excludeFile, null);    
     String[] args = {"-includeFile"};
     String ret = runTool(conf, args, true);
     assertEquals(hostsFile.toUri().getPath(),ret.trim());
-    cleanupFile(localFileSys, excludeFile.getParent());
+    hostsFileWriter.cleanup();
   }
 
   @Test
@@ -443,29 +428,4 @@ public class TestGetConf {
     verifyAddresses(conf, TestType.NAMENODE, false, includedNN);
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, includedNN);
   }
-
-  private void writeConfigFile(Path name, ArrayList<String> nodes) 
-      throws IOException {
-      // delete if it already exists
-      if (localFileSys.exists(name)) {
-        localFileSys.delete(name, true);
-      }
-
-      FSDataOutputStream stm = localFileSys.create(name);
-      
-      if (nodes != null) {
-        for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
-          String node = it.next();
-          stm.writeBytes(node);
-          stm.writeBytes("\n");
-        }
-      }
-      stm.close();
-    }
-  
-  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
-    assertTrue(fileSys.exists(name));
-    fileSys.delete(name, true);
-    assertTrue(!fileSys.exists(name));
-  }
 }

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

@@ -17,6 +17,14 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
+import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.ALL;
+import static org.apache.hadoop.fs.permission.FsAction.EXECUTE;
+import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
+import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -49,6 +57,7 @@ import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
 
 import com.google.common.io.Files;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.output.NullOutputStream;
 import org.apache.commons.logging.Log;
@@ -82,6 +91,7 @@ import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 import org.xml.sax.helpers.DefaultHandler;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 public class TestOfflineImageViewer {
@@ -113,6 +123,7 @@ public class TestOfflineImageViewer {
           DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
       conf.setBoolean(
           DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+      conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
       conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL,
           "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
@@ -179,6 +190,14 @@ public class TestOfflineImageViewer {
       // as UTF8
       hdfs.setXAttr(xattr, "user.a4", new byte[]{ -0x3d, 0x28 });
       writtenFiles.put(xattr.toString(), hdfs.getFileStatus(xattr));
+      // Set ACLs
+      hdfs.setAcl(
+          xattr,
+          Lists.newArrayList(aclEntry(ACCESS, USER, ALL),
+              aclEntry(ACCESS, USER, "foo", ALL),
+              aclEntry(ACCESS, GROUP, READ_EXECUTE),
+              aclEntry(ACCESS, GROUP, "bar", READ_EXECUTE),
+              aclEntry(ACCESS, OTHER, EXECUTE)));
 
       // Write results to the fsimage file
       hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);

+ 25 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/HostsFileWriter.java

@@ -73,16 +73,26 @@ public class HostsFileWriter {
   }
 
   public void initExcludeHost(String hostNameAndPort) throws IOException {
+    initExcludeHosts(hostNameAndPort);
+  }
+
+  public void initExcludeHosts(String... hostNameAndPorts) throws IOException {
+    StringBuilder excludeHosts = new StringBuilder();
     if (isLegacyHostsFile) {
-      DFSTestUtil.writeFile(localFileSys, excludeFile, hostNameAndPort);
+      for (String hostNameAndPort : hostNameAndPorts) {
+        excludeHosts.append(hostNameAndPort).append("\n");
+      }
+      DFSTestUtil.writeFile(localFileSys, excludeFile, excludeHosts.toString());
     } else {
-      DatanodeAdminProperties dn = new DatanodeAdminProperties();
-      String [] hostAndPort = hostNameAndPort.split(":");
-      dn.setHostName(hostAndPort[0]);
-      dn.setPort(Integer.parseInt(hostAndPort[1]));
-      dn.setAdminState(AdminStates.DECOMMISSIONED);
       HashSet<DatanodeAdminProperties> allDNs = new HashSet<>();
-      allDNs.add(dn);
+      for (String hostNameAndPort : hostNameAndPorts) {
+        DatanodeAdminProperties dn = new DatanodeAdminProperties();
+        String[] hostAndPort = hostNameAndPort.split(":");
+        dn.setHostName(hostAndPort[0]);
+        dn.setPort(Integer.parseInt(hostAndPort[1]));
+        dn.setAdminState(AdminStates.DECOMMISSIONED);
+        allDNs.add(dn);
+      }
       CombinedHostsFileWriter.writeFile(combinedFile.toString(), allDNs);
     }
   }
@@ -119,4 +129,12 @@ public class HostsFileWriter {
       FileUtils.deleteQuietly(new File(fullDir.toUri().getPath()));
     }
   }
+
+  public Path getIncludeFile() {
+    return includeFile;
+  }
+
+  public Path getExcludeFile() {
+    return excludeFile;
+  }
 }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java

@@ -23,6 +23,7 @@ import java.io.FileWriter;
 import java.util.Set;
 
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.After;
 import org.junit.Test;
@@ -35,8 +36,8 @@ import static org.junit.Assert.assertEquals;
 public class TestCombinedHostsFileReader {
 
   // Using /test/build/data/tmp directory to store temporary files
-  static final String HOSTS_TEST_DIR = new File(System.getProperty(
-      "test.build.data", "/tmp")).getAbsolutePath();
+  static final String HOSTS_TEST_DIR = GenericTestUtils.getTestDir()
+      .getAbsolutePath();
   File NEW_FILE = new File(HOSTS_TEST_DIR, "dfs.hosts.new.json");
 
   static final String TEST_CACHE_DATA_DIR =

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java

@@ -32,14 +32,15 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestHttpsFileSystem {
-  private static final String BASEDIR = System.getProperty("test.build.dir",
-      "target/test-dir") + "/" + TestHttpsFileSystem.class.getSimpleName();
+  private static final String BASEDIR =
+      GenericTestUtils.getTempPath(TestHttpsFileSystem.class.getSimpleName());
 
   private static MiniDFSCluster cluster;
   private static Configuration conf;

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

@@ -55,6 +55,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -199,8 +200,8 @@ public class TestWebHdfsTokens {
       SecurityUtil.setAuthenticationMethod(SIMPLE, clusterConf);
       clusterConf.setBoolean(DFSConfigKeys
 	    .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
-      String BASEDIR = System.getProperty("test.build.dir",
-	      	  "target/test-dir") + "/" + TestWebHdfsTokens.class.getSimpleName();
+      String baseDir =
+          GenericTestUtils.getTempPath(TestWebHdfsTokens.class.getSimpleName());
       String keystoresDir;
       String sslConfDir;
 	    
@@ -208,10 +209,10 @@ public class TestWebHdfsTokens {
       clusterConf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
       clusterConf.set(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
 	  
-      File base = new File(BASEDIR);
+      File base = new File(baseDir);
       FileUtil.fullyDelete(base);
       base.mkdirs();
-      keystoresDir = new File(BASEDIR).getAbsolutePath();
+      keystoresDir = new File(baseDir).getAbsolutePath();
       sslConfDir = KeyStoreTestUtil.getClasspathDir(TestWebHdfsTokens.class);
       KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, clusterConf, false);
       clusterConf.set(DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,

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

@@ -19,7 +19,6 @@ package org.apache.hadoop.test;
 
 import java.io.File;
 
-import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.fs.Path;
 
 public class PathUtils {
@@ -37,10 +36,8 @@ public class PathUtils {
   }
   
   public static File getTestDir(Class<?> caller, boolean create) {
-    File dir =
-        new File(System.getProperty("test.build.data", "target/test/data")
-            + "/" + RandomStringUtils.randomAlphanumeric(10),
-            caller.getSimpleName());
+    File dir = new File(GenericTestUtils.getRandomizedTestDir(),
+        caller.getSimpleName());
     if (create) {
       dir.mkdirs();
     }

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
@@ -290,11 +291,10 @@ class YarnChild {
   private static void configureTask(JobConf job, Task task,
       Credentials credentials, Token<JobTokenIdentifier> jt) throws IOException {
     job.setCredentials(credentials);
-    
-    ApplicationAttemptId appAttemptId =
-        ConverterUtils.toContainerId(
-            System.getenv(Environment.CONTAINER_ID.name()))
-            .getApplicationAttemptId();
+
+    ApplicationAttemptId appAttemptId = ContainerId.fromString(
+        System.getenv(Environment.CONTAINER_ID.name()))
+        .getApplicationAttemptId();
     LOG.debug("APPLICATION_ATTEMPT_ID: " + appAttemptId);
     // Set it in conf, so as to be able to be used the the OutputCommitter.
     job.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -1562,7 +1562,7 @@ public class MRAppMaster extends CompositeService {
       validateInputParam(appSubmitTimeStr,
           ApplicationConstants.APP_SUBMIT_TIME_ENV);
 
-      ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+      ContainerId containerId = ContainerId.fromString(containerIdStr);
       ApplicationAttemptId applicationAttemptId =
           containerId.getApplicationAttemptId();
       if (applicationAttemptId != null) {

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -713,8 +713,7 @@ public abstract class TaskAttemptImpl implements
       LocalResourceType type, LocalResourceVisibility visibility)
       throws IOException {
     FileStatus fstat = fc.getFileStatus(file);
-    URL resourceURL = ConverterUtils.getYarnUrlFromPath(fc.resolvePath(fstat
-        .getPath()));
+    URL resourceURL = URL.fromPath(fc.resolvePath(fstat.getPath()));
     long resourceSize = fstat.getLen();
     long resourceModificationTime = fstat.getModificationTime();
 
@@ -1247,8 +1246,8 @@ public abstract class TaskAttemptImpl implements
   public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo,
       OutputCommitter committer, boolean recoverOutput) {
     ContainerId containerId = taInfo.getContainerId();
-    NodeId containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":"
-        + taInfo.getPort());
+    NodeId containerNodeId = NodeId.fromString(
+        taInfo.getHostname() + ":" + taInfo.getPort());
     String nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":"
         + taInfo.getHttpPort());
     // Resource/Priority/Tokens are only needed while launching the container on

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java

@@ -69,8 +69,10 @@ public class TaskAttemptInfo {
     this.nodeHttpAddress = ta.getNodeHttpAddress();
     this.startTime = report.getStartTime();
     this.finishTime = report.getFinishTime();
-    this.assignedContainerId = ConverterUtils.toString(report.getContainerId());
     this.assignedContainer = report.getContainerId();
+    if (assignedContainer != null) {
+      this.assignedContainerId = assignedContainer.toString();
+    }
     this.progress = report.getProgress() * 100;
     this.status = report.getStateString();
     this.state = report.getTaskAttemptState();

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -254,7 +254,7 @@ public class MRApp extends MRAppMaster {
     // the job can reaches the final state when MRAppMaster shuts down.
     this.successfullyUnregistered.set(unregistered);
     this.assignedQueue = assignedQueue;
-    this.resource = Resource.newInstance(1234, 2);
+    this.resource = Resource.newInstance(1234L, 2L);
   }
 
   @Override

+ 33 - 33
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java

@@ -114,7 +114,7 @@ public class TestMRAppMaster {
     localFS.delete(testDir, true);
     new File(testDir.toString()).mkdir();
   }
-  
+
   @Before
   public void prepare() throws IOException {
     File dir = new File(stagingDir);
@@ -134,11 +134,11 @@ public class TestMRAppMaster {
       InterruptedException {
     String applicationAttemptIdStr = "appattempt_1317529182569_0004_000001";
     String containerIdStr = "container_1317529182569_0004_000001_1";
-    
+
     String userName = "TestAppMasterUser";
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMasterTest appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis());
@@ -161,15 +161,15 @@ public class TestMRAppMaster {
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     conf.setInt(org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.
         FILEOUTPUTCOMMITTER_ALGORITHM_VERSION, 1);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
     Path start = MRApps.getStartJobCommitFile(conf, userName, jobId);
     FileSystem fs = FileSystem.get(conf);
     //Create the file, but no end file so we should unregister with an error.
     fs.create(start).close();
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -200,8 +200,8 @@ public class TestMRAppMaster {
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     conf.setInt(MRJobConfig.NUM_REDUCES, 0);
     conf.set(JHAdminConfig.MR_HS_JHIST_FORMAT, "json");
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId = TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
 
@@ -219,7 +219,7 @@ public class TestMRAppMaster {
     FileSystem fs = FileSystem.get(conf);
     JobSplitWriter.createSplitFiles(new Path(dir.getAbsolutePath()), conf,
         fs, new org.apache.hadoop.mapred.InputSplit[0]);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMasterTestLaunchTime appMaster =
         new MRAppMasterTestLaunchTime(applicationAttemptId, containerId,
             "host", -1, -1, System.currentTimeMillis());
@@ -237,8 +237,8 @@ public class TestMRAppMaster {
     String userName = "TestAppMasterUser";
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
     Path start = MRApps.getStartJobCommitFile(conf, userName, jobId);
@@ -246,7 +246,7 @@ public class TestMRAppMaster {
     FileSystem fs = FileSystem.get(conf);
     fs.create(start).close();
     fs.create(end).close();
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -266,7 +266,7 @@ public class TestMRAppMaster {
     // verify the final status is SUCCEEDED
     verifyFailedStatus((MRAppMasterTest)appMaster, "SUCCEEDED");
   }
-  
+
   @Test
   public void testMRAppMasterFailLock() throws IOException,
       InterruptedException {
@@ -275,8 +275,8 @@ public class TestMRAppMaster {
     String userName = "TestAppMasterUser";
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(applicationAttemptId.getApplicationId()));
     Path start = MRApps.getStartJobCommitFile(conf, userName, jobId);
@@ -284,7 +284,7 @@ public class TestMRAppMaster {
     FileSystem fs = FileSystem.get(conf);
     fs.create(start).close();
     fs.create(end).close();
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -304,7 +304,7 @@ public class TestMRAppMaster {
     // verify the final status is FAILED
     verifyFailedStatus((MRAppMasterTest)appMaster, "FAILED");
   }
-  
+
   @Test
   public void testMRAppMasterMissingStaging() throws IOException,
       InterruptedException {
@@ -313,16 +313,16 @@ public class TestMRAppMaster {
     String userName = "TestAppMasterUser";
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
 
     //Delete the staging directory
     File dir = new File(stagingDir);
     if(dir.exists()) {
       FileUtils.deleteDirectory(dir);
     }
-    
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     MRAppMaster appMaster =
         new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
             System.currentTimeMillis(), false, false);
@@ -353,9 +353,9 @@ public class TestMRAppMaster {
     String containerIdStr = "container_1317529182569_0004_000002_1";
 
     String userName = "TestAppMasterUser";
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
 
@@ -427,7 +427,7 @@ public class TestMRAppMaster {
         new Token<AMRMTokenIdentifier>(identifier, password,
             AMRMTokenIdentifier.KIND_NAME, appTokenService);
     credentials.addToken(appTokenService, appToken);
-    
+
     Text keyAlias = new Text("mySecretKeyAlias");
     credentials.addSecretKey(keyAlias, "mySecretKey".getBytes());
     Token<? extends TokenIdentifier> storedToken =
@@ -488,7 +488,7 @@ public class TestMRAppMaster {
     Assert.assertEquals(storedToken, confCredentials.getToken(tokenAlias));
     Assert.assertEquals("mySecretKey",
       new String(confCredentials.getSecretKey(keyAlias)));
-    
+
     // Verify the AM's ugi - app token should be present
     Credentials ugiCredentials = appMaster.getUgi().getCredentials();
     Assert.assertEquals(1, ugiCredentials.numberOfSecretKeys());
@@ -507,9 +507,9 @@ public class TestMRAppMaster {
     String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
     String containerIdStr = "container_1317529182569_0004_000002_1";
     String userName = "TestAppMasterUser";
-    ApplicationAttemptId applicationAttemptId = ConverterUtils
-        .toApplicationAttemptId(applicationAttemptIdStr);
-    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.fromString(
+        applicationAttemptIdStr);
+    ContainerId containerId = ContainerId.fromString(containerIdStr);
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
 
@@ -591,7 +591,7 @@ class MRAppMasterTest extends MRAppMaster {
     }
     this.conf = conf;
   }
-  
+
   @Override
   protected ContainerAllocator createContainerAllocator(
       final ClientService clientService, final AppContext context) {
@@ -628,7 +628,7 @@ class MRAppMasterTest extends MRAppMaster {
   public Credentials getCredentials() {
     return super.getCredentials();
   }
-  
+
   public UserGroupInformation getUgi() {
     return currentUser;
   }

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java

@@ -129,8 +129,8 @@ public class TestCommitterEventHandler {
 
     SystemClock clock = SystemClock.getInstance();
     AppContext appContext = mock(AppContext.class);
-    ApplicationAttemptId attemptid = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId attemptid = ApplicationAttemptId.fromString(
+        "appattempt_1234567890000_0001_0");
     when(appContext.getApplicationID()).thenReturn(attemptid.getApplicationId());
     when(appContext.getApplicationAttemptId()).thenReturn(attemptid);
     when(appContext.getEventHandler()).thenReturn(
@@ -240,8 +240,8 @@ public class TestCommitterEventHandler {
     YarnConfiguration conf = new YarnConfiguration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     JobContext mockJobContext = mock(JobContext.class);
-    ApplicationAttemptId attemptid = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId attemptid = ApplicationAttemptId.fromString(
+        "appattempt_1234567890000_0001_0");
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(attemptid.getApplicationId()));
     
@@ -288,8 +288,8 @@ public class TestCommitterEventHandler {
     YarnConfiguration conf = new YarnConfiguration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     JobContext mockJobContext = mock(JobContext.class);
-    ApplicationAttemptId attemptid = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId attemptid =
+        ApplicationAttemptId.fromString("appattempt_1234567890000_0001_0");
     JobId jobId =  TypeConverter.toYarn(
         TypeConverter.fromYarn(attemptid.getApplicationId()));
     

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

@@ -942,8 +942,8 @@ public class TestJobImpl {
         callback.run();
       }
     };
-    ApplicationAttemptId id = 
-      ConverterUtils.toApplicationAttemptId("appattempt_1234567890000_0001_0");
+    ApplicationAttemptId id = ApplicationAttemptId.fromString(
+        "appattempt_1234567890000_0001_0");
     when(appContext.getApplicationID()).thenReturn(id.getApplicationId());
     when(appContext.getApplicationAttemptId()).thenReturn(id);
     CommitterEventHandler handler =

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java

@@ -515,7 +515,7 @@ public class TestAMWebServicesAttempts extends JerseyTest {
     WebServicesTestUtils.checkStringMatch("diagnostics", expectDiag,
         diagnostics);
     WebServicesTestUtils.checkStringMatch("assignedContainerId",
-        ConverterUtils.toString(ta.getAssignedContainerID()),
+        ta.getAssignedContainerID().toString(),
         assignedContainerId);
 
     assertEquals("startTime wrong", ta.getLaunchTime(), startTime);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java

@@ -157,7 +157,7 @@ class LocalDistributedCacheManager {
         }
         Path resourcePath;
         try {
-          resourcePath = ConverterUtils.getPathFromYarnURL(resource.getResource());
+          resourcePath = resource.getResource().toPath();
         } catch (URISyntaxException e) {
           throw new IOException(e);
         }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Apps;
@@ -569,14 +570,13 @@ public class MRApps extends Apps {
         }
         String linkName = name.toUri().getPath();
         LocalResource orig = localResources.get(linkName);
-        if(orig != null && !orig.getResource().equals(
-            ConverterUtils.getYarnUrlFromURI(p.toUri()))) {
+        if(orig != null && !orig.getResource().equals(URL.fromURI(p.toUri()))) {
           throw new InvalidJobConfException(
               getResourceDescription(orig.getType()) + orig.getResource() + 
               " conflicts with " + getResourceDescription(type) + u);
         }
-        localResources.put(linkName, LocalResource.newInstance(ConverterUtils
-          .getYarnUrlFromURI(p.toUri()), type, visibilities[i]
+        localResources.put(linkName, LocalResource
+            .newInstance(URL.fromURI(p.toUri()), type, visibilities[i]
             ? LocalResourceVisibility.PUBLIC : LocalResourceVisibility.PRIVATE,
           sizes[i], timestamps[i]));
       }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java

@@ -107,8 +107,8 @@ public class AMStartedEvent implements HistoryEvent {
    * @return the ApplicationAttemptId
    */
   public ApplicationAttemptId getAppAttemptId() {
-    return ConverterUtils.toApplicationAttemptId(datum.getApplicationAttemptId()
-        .toString());
+    return ApplicationAttemptId.fromString(
+        datum.getApplicationAttemptId().toString());
   }
 
   /**
@@ -122,7 +122,7 @@ public class AMStartedEvent implements HistoryEvent {
    * @return the ContainerId for the MRAppMaster.
    */
   public ContainerId getContainerId() {
-    return ConverterUtils.toContainerId(datum.getContainerId().toString());
+    return ContainerId.fromString(datum.getContainerId().toString());
   }
 
   /**

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java

@@ -75,7 +75,7 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
       long startTime, String trackerName, int httpPort, int shufflePort,
       String locality, String avataar) {
     this(attemptId, taskType, startTime, trackerName, httpPort, shufflePort,
-        ConverterUtils.toContainerId("container_-1_-1_-1_-1"), locality,
+        ContainerId.fromString("container_-1_-1_-1_-1"), locality,
             avataar);
   }
 
@@ -116,7 +116,7 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
   }
   /** Get the ContainerId */
   public ContainerId getContainerId() {
-    return ConverterUtils.toContainerId(datum.getContainerId().toString());
+    return ContainerId.fromString(datum.getContainerId().toString());
   }
   /** Get the locality */
   public String getLocality() {

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java

@@ -53,10 +53,10 @@ public class UncompressedSplitLineReader extends SplitLineReader {
       throws IOException {
     int maxBytesToRead = buffer.length;
     if (totalBytesRead < splitLength) {
-      long leftBytesForSplit = splitLength - totalBytesRead;
-      // check if leftBytesForSplit exceed Integer.MAX_VALUE
-      if (leftBytesForSplit <= Integer.MAX_VALUE) {
-        maxBytesToRead = Math.min(maxBytesToRead, (int)leftBytesForSplit);
+      long bytesLeftInSplit = splitLength - totalBytesRead;
+
+      if (bytesLeftInSplit < maxBytesToRead) {
+        maxBytesToRead = (int)bytesLeftInSplit;
       }
     }
     int bytesRead = in.read(buffer, 0, maxBytesToRead);

برخی فایل ها در این مقایسه diff نمایش داده نمی شوند زیرا تعداد فایل ها بسیار زیاد است