瀏覽代碼

Merge trunk into auto-HA branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3042@1327724 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 年之前
父節點
當前提交
ca49126e3d
共有 100 個文件被更改,包括 2292 次插入588 次删除
  1. 8 13
      dev-support/test-patch.sh
  2. 34 3
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 0 2
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/bin/start-all.sh
  5. 76 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  6. 6 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  7. 9 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  8. 22 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
  9. 20 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java
  10. 42 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  11. 12 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  12. 9 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
  13. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  14. 101 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionUtil.java
  15. 67 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  16. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodec.java
  17. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
  18. 18 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  19. 118 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestServer.java
  20. 30 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java
  21. 0 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  22. 62 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestVersionUtil.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
  24. 16 9
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java
  26. 5 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java
  27. 52 35
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
  28. 0 31
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
  29. 8 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
  30. 17 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java
  31. 70 11
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
  32. 71 6
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java
  33. 6 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java
  34. 48 2
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  35. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh
  37. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
  38. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  39. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  40. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  41. 1 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  42. 19 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  43. 17 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  44. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  45. 20 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
  46. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  47. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  48. 15 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java
  49. 16 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  50. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
  51. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  52. 47 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
  53. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java
  54. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java
  55. 5 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
  56. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  57. 86 34
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  58. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  59. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  60. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  61. 63 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  62. 45 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  63. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  64. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
  65. 96 37
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  66. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
  67. 10 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
  68. 29 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  69. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
  70. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  71. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  72. 118 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
  73. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpDelegationToken.java
  74. 53 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
  75. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
  76. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
  77. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  78. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java
  79. 79 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
  80. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java
  81. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  82. 22 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  83. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  84. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
  85. 68 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
  86. 57 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
  87. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
  88. 54 0
      hadoop-mapreduce-project/CHANGES.txt
  89. 25 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  90. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
  91. 12 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  92. 15 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  93. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  94. 6 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  95. 3 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  96. 2 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
  97. 7 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  98. 3 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfInfo.java
  99. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  100. 7 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

+ 8 - 13
dev-support/test-patch.sh

@@ -627,22 +627,17 @@ runTests () {
   echo ""
   echo ""
   echo ""
   echo ""
 
 
-  echo "$MVN clean install -Pnative -D${PROJECT_NAME}PatchProcess"
-  $MVN clean install -Pnative -D${PROJECT_NAME}PatchProcess
-  if [[ $? != 0 ]] ; then
-    ### Find and format names of failed tests
-    failed_tests=`find . -name 'TEST*.xml' | xargs $GREP  -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-|                  |g" | sed -e "s|\.xml||g"`
-
-    if [[ -n "$failed_tests" ]] ; then
-      JIRA_COMMENT="$JIRA_COMMENT
+  echo "$MVN clean install -fn -Pnative -D${PROJECT_NAME}PatchProcess"
+  $MVN clean install -fn -Pnative -D${PROJECT_NAME}PatchProcess
+  failed_tests=`find . -name 'TEST*.xml' | xargs $GREP  -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-|                  |g" | sed -e "s|\.xml||g"`
+  # With -fn mvn always exits with a 0 exit code.  Because of this we need to
+  # find the errors instead of using the exit code.  We assume that if the build
+  # failed a -1 is already given for that case
+  if [[ -n "$failed_tests" ]] ; then
+    JIRA_COMMENT="$JIRA_COMMENT
 
 
     -1 core tests.  The patch failed these unit tests:
     -1 core tests.  The patch failed these unit tests:
 $failed_tests"
 $failed_tests"
-    else
-      JIRA_COMMENT="$JIRA_COMMENT
-
-    -1 core tests.  The patch failed the unit tests build"
-    fi
     return 1
     return 1
   fi
   fi
   JIRA_COMMENT="$JIRA_COMMENT
   JIRA_COMMENT="$JIRA_COMMENT

+ 34 - 3
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -58,6 +58,11 @@ Trunk (unreleased changes)
     HADOOP-7757. Test file reference count is at least 3x actual value (Jon
     HADOOP-7757. Test file reference count is at least 3x actual value (Jon
     Eagles via bobby)
     Eagles via bobby)
 
 
+    HADOOP-8147. test-patch should run tests with -fn to avoid masking test
+    failures (Robert Evans via tgraves)
+
+    HADOOP-8290. Remove remaining references to hadoop.native.lib (harsh)
+
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.
     HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.
@@ -205,9 +210,6 @@ Release 2.0.0 - UNRELEASED
     HADOOP-7358. Improve log levels when exceptions caught in RPC handler
     HADOOP-7358. Improve log levels when exceptions caught in RPC handler
     (Todd Lipcon via shv)
     (Todd Lipcon via shv)
 
 
-    HADOOP-8108. Move method getHostPortString() from NameNode to NetUtils.
-    (Brandon Li via jitendra)
-
     HADOOP-7557 Make IPC header be extensible (sanjay radia)
     HADOOP-7557 Make IPC header be extensible (sanjay radia)
 
 
     HADOOP-7806. Support binding to sub-interfaces (eli)
     HADOOP-7806. Support binding to sub-interfaces (eli)
@@ -258,6 +260,11 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8086. KerberosName silently sets defaultRealm to "" if the 
     HADOOP-8086. KerberosName silently sets defaultRealm to "" if the 
     Kerberos config is not found, it should log a WARN (tucu)
     Kerberos config is not found, it should log a WARN (tucu)
 
 
+    HADOOP-8280. Move VersionUtil/TestVersionUtil and GenericTestUtils from
+    HDFS into Common. (Ahmed Radwan via atm)
+
+    HADOOP-8117. Upgrade test build to Surefire 2.12 (todd)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -343,6 +350,16 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8264. Remove irritating double double quotes in front of hostname
     HADOOP-8264. Remove irritating double double quotes in front of hostname
     (Bernd Fondermann via bobby)
     (Bernd Fondermann via bobby)
 
 
+    HADOOP-8270. hadoop-daemon.sh stop action should return 0 for an
+    already stopped service. (Roman Shaposhnik via eli)
+
+    HADOOP-8144. pseudoSortByDistance in NetworkTopology doesn't work
+    properly if no local node and first node is local rack node.
+    (Junping Du)
+
+    HADOOP-8282. start-all.sh refers incorrectly start-dfs.sh
+    existence for starting start-yarn.sh. (Devaraj K via eli)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -403,6 +420,9 @@ Release 0.23.3 - UNRELEASED
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    HADOOP-8108. Move method getHostPortString() from NameNode to NetUtils.
+    (Brandon Li via jitendra)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -422,6 +442,17 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8014. ViewFileSystem does not correctly implement getDefaultBlockSize,
     HADOOP-8014. ViewFileSystem does not correctly implement getDefaultBlockSize,
     getDefaultReplication, getContentSummary (John George via bobby)
     getDefaultReplication, getContentSummary (John George via bobby)
 
 
+    HADOOP-7510. Tokens should use original hostname provided instead of ip
+    (Daryn Sharp via bobby)
+
+    HADOOP-8283. Allow tests to control token service value (Daryn Sharp via
+    bobby)
+
+    HADOOP-8286. Simplify getting a socket address from conf (Daryn Sharp via
+    bobby)
+
+    HADOOP-8227. Allow RPC to limit ephemeral port range. (bobby)
+
 Release 0.23.2 - UNRELEASED 
 Release 0.23.2 - UNRELEASED 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh

@@ -167,11 +167,9 @@ case $startStop in
         kill `cat $pid`
         kill `cat $pid`
       else
       else
         echo no $command to stop
         echo no $command to stop
-        exit 1
       fi
       fi
     else
     else
       echo no $command to stop
       echo no $command to stop
-      exit 1
     fi
     fi
     ;;
     ;;
 
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/bin/start-all.sh

@@ -33,6 +33,6 @@ if [ -f "${HADOOP_HDFS_HOME}"/sbin/start-dfs.sh ]; then
 fi
 fi
 
 
 # start yarn daemons if yarn is present
 # start yarn daemons if yarn is present
-if [ -f "${YARN_HOME}"/sbin/start-dfs.sh ]; then
+if [ -f "${YARN_HOME}"/sbin/start-yarn.sh ]; then
   "${YARN_HOME}"/sbin/start-yarn.sh --config $HADOOP_CONF_DIR
   "${YARN_HOME}"/sbin/start-yarn.sh --config $HADOOP_CONF_DIR
 fi
 fi

+ 76 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -30,6 +30,7 @@ import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.OutputStreamWriter;
 import java.io.Reader;
 import java.io.Reader;
 import java.io.Writer;
 import java.io.Writer;
+import java.net.InetSocketAddress;
 import java.net.URL;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
@@ -68,6 +69,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonFactory;
@@ -962,11 +964,57 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * bound may be omitted meaning all values up to or over. So the string 
    * bound may be omitted meaning all values up to or over. So the string 
    * above means 2, 3, 5, and 7, 8, 9, ...
    * above means 2, 3, 5, and 7, 8, 9, ...
    */
    */
-  public static class IntegerRanges {
+  public static class IntegerRanges implements Iterable<Integer>{
     private static class Range {
     private static class Range {
       int start;
       int start;
       int end;
       int end;
     }
     }
+    
+    private static class RangeNumberIterator implements Iterator<Integer> {
+      Iterator<Range> internal;
+      int at;
+      int end;
+
+      public RangeNumberIterator(List<Range> ranges) {
+        if (ranges != null) {
+          internal = ranges.iterator();
+        }
+        at = -1;
+        end = -2;
+      }
+      
+      @Override
+      public boolean hasNext() {
+        if (at <= end) {
+          return true;
+        } else if (internal != null){
+          return internal.hasNext();
+        }
+        return false;
+      }
+
+      @Override
+      public Integer next() {
+        if (at <= end) {
+          at++;
+          return at - 1;
+        } else if (internal != null){
+          Range found = internal.next();
+          if (found != null) {
+            at = found.start;
+            end = found.end;
+            at++;
+            return at - 1;
+          }
+        }
+        return null;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
 
 
     List<Range> ranges = new ArrayList<Range>();
     List<Range> ranges = new ArrayList<Range>();
     
     
@@ -1025,6 +1073,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       return false;
       return false;
     }
     }
     
     
+    /**
+     * @return true if there are no values in this range, else false.
+     */
+    public boolean isEmpty() {
+      return ranges == null || ranges.isEmpty();
+    }
+    
     @Override
     @Override
     public String toString() {
     public String toString() {
       StringBuilder result = new StringBuilder();
       StringBuilder result = new StringBuilder();
@@ -1041,6 +1096,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       }
       }
       return result.toString();
       return result.toString();
     }
     }
+
+    @Override
+    public Iterator<Integer> iterator() {
+      return new RangeNumberIterator(ranges);
+    }
+    
   }
   }
 
 
   /**
   /**
@@ -1162,6 +1223,20 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     set(name, StringUtils.arrayToString(values));
     set(name, StringUtils.arrayToString(values));
   }
   }
 
 
+  /**
+   * Get the socket address for <code>name</code> property as a
+   * <code>InetSocketAddress</code>.
+   * @param name property name.
+   * @param defaultAddress the default value
+   * @param defaultPort the default port
+   * @return InetSocketAddress
+   */
+  public InetSocketAddress getSocketAddr(
+      String name, String defaultAddress, int defaultPort) {
+    final String address = get(name, defaultAddress);
+    return NetUtils.createSocketAddr(address, defaultPort, name);
+  }
+  
   /**
   /**
    * Load a class by name.
    * Load a class by name.
    * 
    * 

+ 6 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.RpcPayloadHeader.*;
 import org.apache.hadoop.ipc.RpcPayloadHeader.*;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -243,8 +242,8 @@ public class Client {
       this.remoteId = remoteId;
       this.remoteId = remoteId;
       this.server = remoteId.getAddress();
       this.server = remoteId.getAddress();
       if (server.isUnresolved()) {
       if (server.isUnresolved()) {
-        throw NetUtils.wrapException(remoteId.getAddress().getHostName(),
-            remoteId.getAddress().getPort(),
+        throw NetUtils.wrapException(server.getHostName(),
+            server.getPort(),
             null,
             null,
             0,
             0,
             new UnknownHostException());
             new UnknownHostException());
@@ -274,9 +273,8 @@ public class Client {
           } catch (IllegalAccessException e) {
           } catch (IllegalAccessException e) {
             throw new IOException(e.toString());
             throw new IOException(e.toString());
           }
           }
-          InetSocketAddress addr = remoteId.getAddress();
-          token = tokenSelector.selectToken(new Text(addr.getAddress()
-              .getHostAddress() + ":" + addr.getPort()), 
+          token = tokenSelector.selectToken(
+              SecurityUtil.buildTokenService(server),
               ticket.getTokens());
               ticket.getTokens());
         }
         }
         KerberosInfo krbInfo = SecurityUtil.getKerberosInfo(protocol, conf);
         KerberosInfo krbInfo = SecurityUtil.getKerberosInfo(protocol, conf);
@@ -305,7 +303,7 @@ public class Client {
             + protocol.getSimpleName());
             + protocol.getSimpleName());
       
       
       this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
       this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
-          remoteId.getAddress().toString() +
+          server.toString() +
           " from " + ((ticket==null)?"an unknown user":ticket.getUserName()));
           " from " + ((ticket==null)?"an unknown user":ticket.getUserName()));
       this.setDaemon(true);
       this.setDaemon(true);
     }
     }
@@ -751,7 +749,6 @@ public class Client {
       }
       }
     }
     }
 
 
-    @SuppressWarnings("unused")
     public InetSocketAddress getRemoteAddress() {
     public InetSocketAddress getRemoteAddress() {
       return server;
       return server;
     }
     }
@@ -1159,7 +1156,7 @@ public class Client {
           call.error.fillInStackTrace();
           call.error.fillInStackTrace();
           throw call.error;
           throw call.error;
         } else { // local exception
         } else { // local exception
-          InetSocketAddress address = remoteId.getAddress();
+          InetSocketAddress address = connection.getRemoteAddress();
           throw NetUtils.wrapException(address.getHostName(),
           throw NetUtils.wrapException(address.getHostName(),
                   address.getPort(),
                   address.getPort(),
                   NetUtils.getHostname(),
                   NetUtils.getHostname(),

+ 9 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -319,10 +319,12 @@ public class ProtobufRpcEngine implements RpcEngine {
   public RPC.Server getServer(Class<?> protocol, Object protocolImpl,
   public RPC.Server getServer(Class<?> protocol, Object protocolImpl,
       String bindAddress, int port, int numHandlers, int numReaders,
       String bindAddress, int port, int numHandlers, int numReaders,
       int queueSizePerHandler, boolean verbose, Configuration conf,
       int queueSizePerHandler, boolean verbose, Configuration conf,
-      SecretManager<? extends TokenIdentifier> secretManager)
+      SecretManager<? extends TokenIdentifier> secretManager,
+      String portRangeConfig)
       throws IOException {
       throws IOException {
     return new Server(protocol, protocolImpl, conf, bindAddress, port,
     return new Server(protocol, protocolImpl, conf, bindAddress, port,
-        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager);
+        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager,
+        portRangeConfig);
   }
   }
   
   
   public static class Server extends RPC.Server {
   public static class Server extends RPC.Server {
@@ -336,15 +338,18 @@ public class ProtobufRpcEngine implements RpcEngine {
      * @param port the port to listen for connections on
      * @param port the port to listen for connections on
      * @param numHandlers the number of method handler threads to run
      * @param numHandlers the number of method handler threads to run
      * @param verbose whether each call should be logged
      * @param verbose whether each call should be logged
+     * @param portRangeConfig A config parameter that can be used to restrict
+     * the range of ports used when port is 0 (an ephemeral port)
      */
      */
     public Server(Class<?> protocolClass, Object protocolImpl,
     public Server(Class<?> protocolClass, Object protocolImpl,
         Configuration conf, String bindAddress, int port, int numHandlers,
         Configuration conf, String bindAddress, int port, int numHandlers,
         int numReaders, int queueSizePerHandler, boolean verbose,
         int numReaders, int queueSizePerHandler, boolean verbose,
-        SecretManager<? extends TokenIdentifier> secretManager)
+        SecretManager<? extends TokenIdentifier> secretManager, 
+        String portRangeConfig)
         throws IOException {
         throws IOException {
       super(bindAddress, port, null, numHandlers,
       super(bindAddress, port, null, numHandlers,
           numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
           numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
-              .getClass().getName()), secretManager);
+              .getClass().getName()), secretManager, portRangeConfig);
       this.verbose = verbose;  
       this.verbose = verbose;  
       registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
       registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
           protocolImpl);
           protocolImpl);

+ 22 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java

@@ -654,7 +654,8 @@ public class RPC {
                                  final boolean verbose, Configuration conf) 
                                  final boolean verbose, Configuration conf) 
     throws IOException {
     throws IOException {
     return getServer(instance.getClass(),         // use impl class for protocol
     return getServer(instance.getClass(),         // use impl class for protocol
-                     instance, bindAddress, port, numHandlers, false, conf, null);
+                     instance, bindAddress, port, numHandlers, false, conf, null,
+                     null);
   }
   }
 
 
   /** Construct a server for a protocol implementation instance. */
   /** Construct a server for a protocol implementation instance. */
@@ -662,7 +663,8 @@ public class RPC {
                                  Object instance, String bindAddress,
                                  Object instance, String bindAddress,
                                  int port, Configuration conf) 
                                  int port, Configuration conf) 
     throws IOException {
     throws IOException {
-    return getServer(protocol, instance, bindAddress, port, 1, false, conf, null);
+    return getServer(protocol, instance, bindAddress, port, 1, false, conf, null,
+        null);
   }
   }
 
 
   /** Construct a server for a protocol implementation instance.
   /** Construct a server for a protocol implementation instance.
@@ -676,7 +678,7 @@ public class RPC {
     throws IOException {
     throws IOException {
     
     
     return getServer(protocol, instance, bindAddress, port, numHandlers, verbose,
     return getServer(protocol, instance, bindAddress, port, numHandlers, verbose,
-                 conf, null);
+                 conf, null, null);
   }
   }
   
   
   /** Construct a server for a protocol implementation instance. */
   /** Construct a server for a protocol implementation instance. */
@@ -686,10 +688,20 @@ public class RPC {
                                  boolean verbose, Configuration conf,
                                  boolean verbose, Configuration conf,
                                  SecretManager<? extends TokenIdentifier> secretManager) 
                                  SecretManager<? extends TokenIdentifier> secretManager) 
     throws IOException {
     throws IOException {
-    
+    return getServer(protocol, instance, bindAddress, port, numHandlers, verbose,
+        conf, secretManager, null);
+  }
+  
+  public static Server getServer(Class<?> protocol,
+      Object instance, String bindAddress, int port,
+      int numHandlers,
+      boolean verbose, Configuration conf,
+      SecretManager<? extends TokenIdentifier> secretManager,
+      String portRangeConfig) 
+  throws IOException {
     return getProtocolEngine(protocol, conf)
     return getProtocolEngine(protocol, conf)
       .getServer(protocol, instance, bindAddress, port, numHandlers, -1, -1,
       .getServer(protocol, instance, bindAddress, port, numHandlers, -1, -1,
-                 verbose, conf, secretManager);
+                 verbose, conf, secretManager, portRangeConfig);
   }
   }
 
 
   /** Construct a server for a protocol implementation instance. */
   /** Construct a server for a protocol implementation instance. */
@@ -704,7 +716,8 @@ public class RPC {
     
     
     return getProtocolEngine(protocol, conf)
     return getProtocolEngine(protocol, conf)
       .getServer(protocol, instance, bindAddress, port, numHandlers,
       .getServer(protocol, instance, bindAddress, port, numHandlers,
-                 numReaders, queueSizePerHandler, verbose, conf, secretManager);
+                 numReaders, queueSizePerHandler, verbose, conf, secretManager,
+                 null);
   }
   }
 
 
   /** An RPC Server. */
   /** An RPC Server. */
@@ -855,9 +868,10 @@ public class RPC {
                      Class<? extends Writable> paramClass, int handlerCount,
                      Class<? extends Writable> paramClass, int handlerCount,
                      int numReaders, int queueSizePerHandler,
                      int numReaders, int queueSizePerHandler,
                      Configuration conf, String serverName, 
                      Configuration conf, String serverName, 
-                     SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
+                     SecretManager<? extends TokenIdentifier> secretManager,
+                     String portRangeConfig) throws IOException {
       super(bindAddress, port, paramClass, handlerCount, numReaders, queueSizePerHandler,
       super(bindAddress, port, paramClass, handlerCount, numReaders, queueSizePerHandler,
-            conf, serverName, secretManager);
+            conf, serverName, secretManager, portRangeConfig);
       initProtocolMetaInfo(conf);
       initProtocolMetaInfo(conf);
     }
     }
     
     

+ 20 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java

@@ -47,12 +47,30 @@ public interface RpcEngine {
                 UserGroupInformation ticket, Configuration conf)
                 UserGroupInformation ticket, Configuration conf)
     throws IOException, InterruptedException;
     throws IOException, InterruptedException;
 
 
-  /** Construct a server for a protocol implementation instance. */
+  /** 
+   * Construct a server for a protocol implementation instance.
+   * 
+   * @param protocol the class of protocol to use
+   * @param instance the instance of protocol whose methods will be called
+   * @param conf the configuration to use
+   * @param bindAddress the address to bind on to listen for connection
+   * @param port the port to listen for connections on
+   * @param numHandlers the number of method handler threads to run
+   * @param numReaders the number of reader threads to run
+   * @param queueSizePerHandler the size of the queue per hander thread
+   * @param verbose whether each call should be logged
+   * @param secretManager The secret manager to use to validate incoming requests.
+   * @param portRangeConfig A config parameter that can be used to restrict
+   *        the range of ports used when port is 0 (an ephemeral port)
+   * @return The Server instance
+   * @throws IOException on any error
+   */
   RPC.Server getServer(Class<?> protocol, Object instance, String bindAddress,
   RPC.Server getServer(Class<?> protocol, Object instance, String bindAddress,
                        int port, int numHandlers, int numReaders,
                        int port, int numHandlers, int numReaders,
                        int queueSizePerHandler, boolean verbose,
                        int queueSizePerHandler, boolean verbose,
                        Configuration conf, 
                        Configuration conf, 
-                       SecretManager<? extends TokenIdentifier> secretManager
+                       SecretManager<? extends TokenIdentifier> secretManager,
+                       String portRangeConfig
                        ) throws IOException;
                        ) throws IOException;
 
 
   /**
   /**

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

@@ -63,6 +63,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.BytesWritable;
@@ -291,6 +292,7 @@ public abstract class Server {
   protected RpcDetailedMetrics rpcDetailedMetrics;
   protected RpcDetailedMetrics rpcDetailedMetrics;
   
   
   private Configuration conf;
   private Configuration conf;
+  private String portRangeConfig = null;
   private SecretManager<TokenIdentifier> secretManager;
   private SecretManager<TokenIdentifier> secretManager;
   private ServiceAuthorizationManager serviceAuthorizationManager = new ServiceAuthorizationManager();
   private ServiceAuthorizationManager serviceAuthorizationManager = new ServiceAuthorizationManager();
 
 
@@ -323,8 +325,33 @@ public abstract class Server {
    */
    */
   public static void bind(ServerSocket socket, InetSocketAddress address, 
   public static void bind(ServerSocket socket, InetSocketAddress address, 
                           int backlog) throws IOException {
                           int backlog) throws IOException {
+    bind(socket, address, backlog, null, null);
+  }
+
+  public static void bind(ServerSocket socket, InetSocketAddress address, 
+      int backlog, Configuration conf, String rangeConf) throws IOException {
     try {
     try {
-      socket.bind(address, backlog);
+      IntegerRanges range = null;
+      if (rangeConf != null) {
+        range = conf.getRange(rangeConf, "");
+      }
+      if (range == null || range.isEmpty() || (address.getPort() != 0)) {
+        socket.bind(address, backlog);
+      } else {
+        for (Integer port : range) {
+          if (socket.isBound()) break;
+          try {
+            InetSocketAddress temp = new InetSocketAddress(address.getAddress(),
+                port);
+            socket.bind(temp, backlog);
+          } catch(BindException e) {
+            //Ignored
+          }
+        }
+        if (!socket.isBound()) {
+          throw new BindException("Could not find a free port in "+range);
+        }
+      }
     } catch (SocketException e) {
     } catch (SocketException e) {
       throw NetUtils.wrapException(null,
       throw NetUtils.wrapException(null,
           0,
           0,
@@ -424,7 +451,7 @@ public abstract class Server {
       acceptChannel.configureBlocking(false);
       acceptChannel.configureBlocking(false);
 
 
       // Bind the server socket to the local host and port
       // Bind the server socket to the local host and port
-      bind(acceptChannel.socket(), address, backlogLength);
+      bind(acceptChannel.socket(), address, backlogLength, conf, portRangeConfig);
       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
       // create a selector;
       // create a selector;
       selector= Selector.open();
       selector= Selector.open();
@@ -1725,7 +1752,16 @@ public abstract class Server {
     throws IOException 
     throws IOException 
   {
   {
     this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer
     this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer
-        .toString(port), null);
+        .toString(port), null, null);
+  }
+  
+  protected Server(String bindAddress, int port,
+      Class<? extends Writable> rpcRequestClass, int handlerCount,
+      int numReaders, int queueSizePerHandler, Configuration conf,
+      String serverName, SecretManager<? extends TokenIdentifier> secretManager)
+    throws IOException {
+    this(bindAddress, port, rpcRequestClass, handlerCount, numReaders, 
+        queueSizePerHandler, conf, serverName, secretManager, null);
   }
   }
   
   
   /** 
   /** 
@@ -1745,10 +1781,12 @@ public abstract class Server {
   protected Server(String bindAddress, int port,
   protected Server(String bindAddress, int port,
       Class<? extends Writable> rpcRequestClass, int handlerCount,
       Class<? extends Writable> rpcRequestClass, int handlerCount,
       int numReaders, int queueSizePerHandler, Configuration conf,
       int numReaders, int queueSizePerHandler, Configuration conf,
-      String serverName, SecretManager<? extends TokenIdentifier> secretManager)
+      String serverName, SecretManager<? extends TokenIdentifier> secretManager,
+      String portRangeConfig)
     throws IOException {
     throws IOException {
     this.bindAddress = bindAddress;
     this.bindAddress = bindAddress;
     this.conf = conf;
     this.conf = conf;
+    this.portRangeConfig = portRangeConfig;
     this.port = port;
     this.port = port;
     this.rpcRequestClass = rpcRequestClass; 
     this.rpcRequestClass = rpcRequestClass; 
     this.handlerCount = handlerCount;
     this.handlerCount = handlerCount;

+ 12 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -299,16 +299,19 @@ public class WritableRpcEngine implements RpcEngine {
     }
     }
   }
   }
 
 
-  /** Construct a server for a protocol implementation instance listening on a
+  /* Construct a server for a protocol implementation instance listening on a
    * port and address. */
    * port and address. */
+  @Override
   public RPC.Server getServer(Class<?> protocolClass,
   public RPC.Server getServer(Class<?> protocolClass,
                       Object protocolImpl, String bindAddress, int port,
                       Object protocolImpl, String bindAddress, int port,
                       int numHandlers, int numReaders, int queueSizePerHandler,
                       int numHandlers, int numReaders, int queueSizePerHandler,
                       boolean verbose, Configuration conf,
                       boolean verbose, Configuration conf,
-                      SecretManager<? extends TokenIdentifier> secretManager) 
+                      SecretManager<? extends TokenIdentifier> secretManager,
+                      String portRangeConfig) 
     throws IOException {
     throws IOException {
     return new Server(protocolClass, protocolImpl, conf, bindAddress, port,
     return new Server(protocolClass, protocolImpl, conf, bindAddress, port,
-        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager);
+        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager,
+        portRangeConfig);
   }
   }
 
 
 
 
@@ -341,7 +344,7 @@ public class WritableRpcEngine implements RpcEngine {
         Configuration conf, String bindAddress, int port) 
         Configuration conf, String bindAddress, int port) 
       throws IOException {
       throws IOException {
       this(protocolClass, protocolImpl, conf,  bindAddress, port, 1, -1, -1,
       this(protocolClass, protocolImpl, conf,  bindAddress, port, 1, -1, -1,
-          false, null);
+          false, null, null);
     }
     }
     
     
     /** 
     /** 
@@ -363,7 +366,7 @@ public class WritableRpcEngine implements RpcEngine {
             throws IOException {
             throws IOException {
        this(null, protocolImpl,  conf,  bindAddress,   port,
        this(null, protocolImpl,  conf,  bindAddress,   port,
                    numHandlers,  numReaders,  queueSizePerHandler,  verbose, 
                    numHandlers,  numReaders,  queueSizePerHandler,  verbose, 
-                   secretManager);
+                   secretManager, null);
    
    
     }
     }
     
     
@@ -381,11 +384,13 @@ public class WritableRpcEngine implements RpcEngine {
     public Server(Class<?> protocolClass, Object protocolImpl,
     public Server(Class<?> protocolClass, Object protocolImpl,
         Configuration conf, String bindAddress,  int port,
         Configuration conf, String bindAddress,  int port,
         int numHandlers, int numReaders, int queueSizePerHandler, 
         int numHandlers, int numReaders, int queueSizePerHandler, 
-        boolean verbose, SecretManager<? extends TokenIdentifier> secretManager) 
+        boolean verbose, SecretManager<? extends TokenIdentifier> secretManager,
+        String portRangeConfig) 
         throws IOException {
         throws IOException {
       super(bindAddress, port, null, numHandlers, numReaders,
       super(bindAddress, port, null, numHandlers, numReaders,
           queueSizePerHandler, conf,
           queueSizePerHandler, conf,
-          classNameBase(protocolImpl.getClass().getName()), secretManager);
+          classNameBase(protocolImpl.getClass().getName()), secretManager,
+          portRangeConfig);
 
 
       this.verbose = verbose;
       this.verbose = verbose;
       
       

+ 9 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java

@@ -344,8 +344,8 @@ public class NetUtils {
   /**
   /**
    * Returns InetSocketAddress that a client can use to 
    * Returns InetSocketAddress that a client can use to 
    * connect to the server. Server.getListenerAddress() is not correct when
    * connect to the server. Server.getListenerAddress() is not correct when
-   * the server binds to "0.0.0.0". This returns "127.0.0.1:port" when
-   * the getListenerAddress() returns "0.0.0.0:port".
+   * the server binds to "0.0.0.0". This returns "hostname:port" of the server,
+   * or "127.0.0.1:port" when the getListenerAddress() returns "0.0.0.0:port".
    * 
    * 
    * @param server
    * @param server
    * @return socket address that a client can use to connect to the server.
    * @return socket address that a client can use to connect to the server.
@@ -353,7 +353,12 @@ public class NetUtils {
   public static InetSocketAddress getConnectAddress(Server server) {
   public static InetSocketAddress getConnectAddress(Server server) {
     InetSocketAddress addr = server.getListenerAddress();
     InetSocketAddress addr = server.getListenerAddress();
     if (addr.getAddress().isAnyLocalAddress()) {
     if (addr.getAddress().isAnyLocalAddress()) {
-      addr = createSocketAddrForHost("127.0.0.1", addr.getPort());
+      try {
+        addr = new InetSocketAddress(InetAddress.getLocalHost(), addr.getPort());
+      } catch (UnknownHostException uhe) {
+        // shouldn't get here unless the host doesn't have a loopback iface
+        addr = createSocketAddrForHost("127.0.0.1", addr.getPort());
+      }
     }
     }
     return addr;
     return addr;
   }
   }
@@ -655,7 +660,7 @@ public class NetUtils {
     }
     }
     InetAddress addr = null;
     InetAddress addr = null;
     try {
     try {
-      addr = InetAddress.getByName(host);
+      addr = SecurityUtil.getByName(host);
       if (NetworkInterface.getByInetAddress(addr) == null) {
       if (NetworkInterface.getByInetAddress(addr) == null) {
         addr = null; // Not a local address
         addr = null; // Not a local address
       }
       }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -662,8 +662,8 @@ public class NetworkTopology {
    */
    */
   public void pseudoSortByDistance( Node reader, Node[] nodes ) {
   public void pseudoSortByDistance( Node reader, Node[] nodes ) {
     int tempIndex = 0;
     int tempIndex = 0;
+    int localRackNode = -1;
     if (reader != null ) {
     if (reader != null ) {
-      int localRackNode = -1;
       //scan the array to find the local node & local rack node
       //scan the array to find the local node & local rack node
       for(int i=0; i<nodes.length; i++) {
       for(int i=0; i<nodes.length; i++) {
         if(tempIndex == 0 && reader == nodes[i]) { //local node
         if(tempIndex == 0 && reader == nodes[i]) { //local node
@@ -693,7 +693,7 @@ public class NetworkTopology {
     }
     }
     
     
     // put a random node at position 0 if it is not a local/local-rack node
     // put a random node at position 0 if it is not a local/local-rack node
-    if(tempIndex == 0 && nodes.length != 0) {
+    if(tempIndex == 0 && localRackNode == -1 && nodes.length != 0) {
       swap(nodes, 0, r.nextInt(nodes.length));
       swap(nodes, 0, r.nextInt(nodes.length));
     }
     }
   }
   }

+ 101 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionUtil.java

@@ -0,0 +1,101 @@
+/**
+ * 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.util;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public abstract class VersionUtil {
+  
+  private static final Pattern COMPONENT_GROUPS = Pattern.compile("(\\d+)|(\\D+)");
+
+  /**
+   * This function splits the two versions on &quot;.&quot; and performs a
+   * naturally-ordered comparison of the resulting components. For example, the
+   * version string "0.3" is considered to precede "0.20", despite the fact that
+   * lexical comparison would consider "0.20" to precede "0.3". This method of
+   * comparison is similar to the method used by package versioning systems like
+   * deb and RPM.
+   * 
+   * Version components are compared numerically whenever possible, however a
+   * version component can contain non-numeric characters. When a non-numeric
+   * group of characters is found in a version component, this group is compared
+   * with the similarly-indexed group in the other version component. If the
+   * other group is numeric, then the numeric group is considered to precede the
+   * non-numeric group. If both groups are non-numeric, then a lexical
+   * comparison is performed.
+   * 
+   * If two versions have a different number of components, then only the lower
+   * number of components are compared. If those components are identical
+   * between the two versions, then the version with fewer components is
+   * considered to precede the version with more components.
+   * 
+   * This function returns a negative integer if version1 precedes version2, a
+   * positive integer if version2 precedes version1, and 0 if and only if the
+   * two versions' components are identical in value and cardinality.
+   * 
+   * @param version1
+   *          the first version to compare
+   * @param version2
+   *          the second version to compare
+   * @return a negative integer if version1 precedes version2, a positive
+   *         integer if version2 precedes version1, and 0 if and only if the two
+   *         versions are equal.
+   */
+  public static int compareVersions(String version1, String version2) {
+    String[] version1Parts = version1.split("\\.");
+    String[] version2Parts = version2.split("\\.");
+    
+    for (int i = 0; i < version1Parts.length && i < version2Parts.length; i++) {
+      String component1 = version1Parts[i];
+      String component2 = version2Parts[i];
+      if (!component1.equals(component2)) {
+        Matcher matcher1 = COMPONENT_GROUPS.matcher(component1);
+        Matcher matcher2 = COMPONENT_GROUPS.matcher(component2);
+        
+        while (matcher1.find() && matcher2.find()) {
+          String group1 = matcher1.group();
+          String group2 = matcher2.group();
+          if (!group1.equals(group2)) {
+            if (isNumeric(group1) && isNumeric(group2)) {
+              return Integer.parseInt(group1) - Integer.parseInt(group2);
+            } else if (!isNumeric(group1) && !isNumeric(group2)) {
+              return group1.compareTo(group2);
+            } else {
+              return isNumeric(group1) ? -1 : 1;
+            }
+          }
+        }
+        return component1.length() - component2.length();
+      }
+    }
+    return version1Parts.length - version2Parts.length;
+  }
+  
+  private static boolean isNumeric(String s) {
+    try {
+      Integer.parseInt(s);
+      return true;
+    } catch (NumberFormatException nfe) {
+      return false;
+    }
+  }
+}

+ 67 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -23,18 +23,24 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.io.StringWriter;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Map;
 import java.util.Random;
 import java.util.Random;
+import java.util.Set;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertArrayEquals;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
 import org.codehaus.jackson.map.ObjectMapper; 
 import org.codehaus.jackson.map.ObjectMapper; 
 
 
 public class TestConfiguration extends TestCase {
 public class TestConfiguration extends TestCase {
@@ -360,6 +366,35 @@ public class TestConfiguration extends TestCase {
     assertEquals(true, range.isIncluded(34));
     assertEquals(true, range.isIncluded(34));
     assertEquals(true, range.isIncluded(100000000));
     assertEquals(true, range.isIncluded(100000000));
   }
   }
+  
+  public void testGetRangeIterator() throws Exception {
+    Configuration config = new Configuration(false);
+    IntegerRanges ranges = config.getRange("Test", "");
+    assertFalse("Empty range has values", ranges.iterator().hasNext());
+    ranges = config.getRange("Test", "5");
+    Set<Integer> expected = new HashSet<Integer>(Arrays.asList(5));
+    Set<Integer> found = new HashSet<Integer>();
+    for(Integer i: ranges) {
+      found.add(i);
+    }
+    assertEquals(expected, found);
+
+    ranges = config.getRange("Test", "5-10,13-14");
+    expected = new HashSet<Integer>(Arrays.asList(5,6,7,8,9,10,13,14));
+    found = new HashSet<Integer>();
+    for(Integer i: ranges) {
+      found.add(i);
+    }
+    assertEquals(expected, found);
+    
+    ranges = config.getRange("Test", "8-12, 5- 7");
+    expected = new HashSet<Integer>(Arrays.asList(5,6,7,8,9,10,11,12));
+    found = new HashSet<Integer>();
+    for(Integer i: ranges) {
+      found.add(i);
+    }
+    assertEquals(expected, found);
+  }
 
 
   public void testHexValues() throws IOException{
   public void testHexValues() throws IOException{
     out=new BufferedWriter(new FileWriter(CONFIG));
     out=new BufferedWriter(new FileWriter(CONFIG));
@@ -604,6 +639,38 @@ public class TestConfiguration extends TestCase {
                  conf.getPattern("test.pattern3", defaultPattern).pattern());
                  conf.getPattern("test.pattern3", defaultPattern).pattern());
   }
   }
 
 
+  public void testSocketAddress() throws IOException {
+    Configuration conf = new Configuration();
+    final String defaultAddr = "host:1";
+    final int defaultPort = 2;
+    InetSocketAddress addr = null;
+    
+    addr = conf.getSocketAddr("myAddress", defaultAddr, defaultPort);
+    assertEquals(defaultAddr, NetUtils.getHostPortString(addr));
+    
+    conf.set("myAddress", "host2");
+    addr = conf.getSocketAddr("myAddress", defaultAddr, defaultPort);
+    assertEquals("host2:"+defaultPort, NetUtils.getHostPortString(addr));
+    
+    conf.set("myAddress", "host2:3");
+    addr = conf.getSocketAddr("myAddress", defaultAddr, defaultPort);
+    assertEquals("host2:3", NetUtils.getHostPortString(addr));
+    
+    boolean threwException = false;
+    conf.set("myAddress", "bad:-port");
+    try {
+      addr = conf.getSocketAddr("myAddress", defaultAddr, defaultPort);
+    } catch (IllegalArgumentException iae) {
+      threwException = true;
+      assertEquals("Does not contain a valid host:port authority: " +
+                   "bad:-port (configuration property 'myAddress')",
+                   iae.getMessage());
+      
+    } finally {
+      assertTrue(threwException);
+    }
+  }
+
   public void testReload() throws IOException {
   public void testReload() throws IOException {
     out=new BufferedWriter(new FileWriter(CONFIG));
     out=new BufferedWriter(new FileWriter(CONFIG));
     startConfig();
     startConfig();

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodec.java

@@ -756,7 +756,8 @@ public class TestCodec {
 
 
     // Don't use native libs for this test.
     // Don't use native libs for this test.
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
-    conf.setBoolean("hadoop.native.lib", false);
+    conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
+                    false);
     assertFalse("ZlibFactory is using native libs against request",
     assertFalse("ZlibFactory is using native libs against request",
                 ZlibFactory.isNativeZlibLoaded(conf));
                 ZlibFactory.isNativeZlibLoaded(conf));
 
 

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java

@@ -33,6 +33,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -237,7 +238,8 @@ public class TestTFileSeqFileComparison extends TestCase {
     public SeqFileAppendable(FileSystem fs, Path path, int osBufferSize,
     public SeqFileAppendable(FileSystem fs, Path path, int osBufferSize,
         String compress, int minBlkSize) throws IOException {
         String compress, int minBlkSize) throws IOException {
       Configuration conf = new Configuration();
       Configuration conf = new Configuration();
-      conf.setBoolean("hadoop.native.lib", true);
+      conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
+                      true);
 
 
       CompressionCodec codec = null;
       CompressionCodec codec = null;
       if ("lzo".equals(compress)) {
       if ("lzo".equals(compress)) {

+ 18 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.ipc;
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.ConnectException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadInfo;
@@ -41,6 +42,8 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.TestSaslRPC.TestSaslImpl;
+import org.apache.hadoop.ipc.TestSaslRPC.TestSaslProtocol;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -257,7 +260,8 @@ public class TestRPC {
     public org.apache.hadoop.ipc.RPC.Server getServer(Class<?> protocol,
     public org.apache.hadoop.ipc.RPC.Server getServer(Class<?> protocol,
         Object instance, String bindAddress, int port, int numHandlers,
         Object instance, String bindAddress, int port, int numHandlers,
         int numReaders, int queueSizePerHandler, boolean verbose, Configuration conf,
         int numReaders, int queueSizePerHandler, boolean verbose, Configuration conf,
-        SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
+        SecretManager<? extends TokenIdentifier> secretManager, 
+        String portRangeConfig) throws IOException {
       return null;
       return null;
     }
     }
 
 
@@ -542,6 +546,19 @@ public class TestRPC {
     }
     }
   }
   }
   
   
+  @Test
+  public void testServerAddress() throws IOException {
+    Server server = RPC.getServer(TestProtocol.class,
+        new TestImpl(), ADDRESS, 0, 5, true, conf, null);
+    InetSocketAddress bindAddr = null;
+    try {
+      bindAddr = NetUtils.getConnectAddress(server);
+    } finally {
+      server.stop();
+    }
+    assertEquals(bindAddr.getAddress(), InetAddress.getLocalHost());
+  }
+  
   @Test
   @Test
   public void testAuthorization() throws Exception {
   public void testAuthorization() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();

+ 118 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestServer.java

@@ -0,0 +1,118 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.*;
+
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+/**
+ * This is intended to be a set of unit tests for the 
+ * org.apache.hadoop.ipc.Server class.
+ */
+public class TestServer {
+
+  @Test
+  public void testBind() throws Exception {
+    Configuration conf = new Configuration();
+    ServerSocket socket = new ServerSocket();
+    InetSocketAddress address = new InetSocketAddress("0.0.0.0",0);
+    socket.bind(address);
+    try {
+      int min = socket.getLocalPort();
+      int max = min + 100;
+      conf.set("TestRange", min+"-"+max);
+      
+
+      ServerSocket socket2 = new ServerSocket();
+      InetSocketAddress address2 = new InetSocketAddress("0.0.0.0", 0);
+      Server.bind(socket2, address2, 10, conf, "TestRange");
+      try {
+        assertTrue(socket2.isBound());
+        assertTrue(socket2.getLocalPort() > min);
+        assertTrue(socket2.getLocalPort() <= max);
+      } finally {
+        socket2.close();
+      }
+    } finally {
+      socket.close();
+    }
+  }
+  
+  @Test
+  public void testBindSimple() throws Exception {
+    ServerSocket socket = new ServerSocket();
+    InetSocketAddress address = new InetSocketAddress("0.0.0.0",0);
+    Server.bind(socket, address, 10);
+    try {
+      assertTrue(socket.isBound());
+    } finally {
+      socket.close();
+    }
+  }
+
+  @Test
+  public void testEmptyConfig() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("TestRange", "");
+
+
+    ServerSocket socket = new ServerSocket();
+    InetSocketAddress address = new InetSocketAddress("0.0.0.0", 0);
+    try {
+      Server.bind(socket, address, 10, conf, "TestRange");
+      assertTrue(socket.isBound());
+    } finally {
+      socket.close();
+    }
+  }
+  
+  
+  @Test
+  public void testBindError() throws Exception {
+    Configuration conf = new Configuration();
+    ServerSocket socket = new ServerSocket();
+    InetSocketAddress address = new InetSocketAddress("0.0.0.0",0);
+    socket.bind(address);
+    try {
+      int min = socket.getLocalPort();
+      conf.set("TestRange", min+"-"+min);
+      
+
+      ServerSocket socket2 = new ServerSocket();
+      InetSocketAddress address2 = new InetSocketAddress("0.0.0.0", 0);
+      boolean caught = false;
+      try {
+        Server.bind(socket2, address2, 10, conf, "TestRange");
+      } catch (BindException e) {
+        caught = true;
+      } finally {
+        socket2.close();
+      }
+      assertTrue("Failed to catch the expected bind exception",caught);
+    } finally {
+      socket.close();
+    }
+  }
+}

+ 30 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/SecurityUtilTestHelper.java

@@ -0,0 +1,30 @@
+/**
+ * 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.security;
+
+/** helper utils for tests */
+public class SecurityUtilTestHelper {
+  
+  /**
+   * Allow tests to change the resolver used for tokens
+   * @param flag boolean for whether token services use ips or hosts
+   */
+  public static void setTokenServiceUseIp(boolean flag) {
+    SecurityUtil.setTokenServiceUseIp(flag);
+  }
+}

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/GenericTestUtils.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java


+ 62 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestVersionUtil.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+public class TestVersionUtil {
+
+  @Test
+  public void testCompareVersions() {
+    // Equal versions are equal.
+    assertEquals(0, VersionUtil.compareVersions("2.0.0", "2.0.0"));
+    assertEquals(0, VersionUtil.compareVersions("2.0.0a", "2.0.0a"));
+    assertEquals(0, VersionUtil.compareVersions("1", "1"));
+    
+    // Assert that lower versions are lower, and higher versions are higher.
+    assertExpectedValues("1", "2.0.0");
+    assertExpectedValues("1.0.0", "2");
+    assertExpectedValues("1.0.0", "2.0.0");
+    assertExpectedValues("1.0", "2.0.0");
+    assertExpectedValues("1.0.0", "2.0.0");
+    assertExpectedValues("1.0.0", "1.0.0a");
+    assertExpectedValues("1.0.0.0", "2.0.0");
+    assertExpectedValues("1.0.0", "1.0.0-dev");
+    assertExpectedValues("1.0.0", "1.0.1");
+    assertExpectedValues("1.0.0", "1.0.2");
+    assertExpectedValues("1.0.0", "1.1.0");
+    assertExpectedValues("2.0.0", "10.0.0");
+    assertExpectedValues("1.0.0", "1.0.0a");
+    assertExpectedValues("1.0.2a", "1.0.10");
+    assertExpectedValues("1.0.2a", "1.0.2b");
+    assertExpectedValues("1.0.2a", "1.0.2ab");
+    assertExpectedValues("1.0.0a1", "1.0.0a2");
+    assertExpectedValues("1.0.0a2", "1.0.0a10");
+    assertExpectedValues("1.0", "1.a");
+    assertExpectedValues("1.0", "1.a0");
+  }
+  
+  private static void assertExpectedValues(String lower, String higher) {
+    assertTrue(VersionUtil.compareVersions(lower, higher) < 0);
+    assertTrue(VersionUtil.compareVersions(higher, lower) > 0);
+  }
+  
+}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java

@@ -170,7 +170,7 @@ public class HttpFSServer {
     throws IOException, FileSystemAccessException {
     throws IOException, FileSystemAccessException {
     String hadoopUser = getEffectiveUser(user, doAs);
     String hadoopUser = getEffectiveUser(user, doAs);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
-    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getDefaultConfiguration();
+    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getFileSystemConfiguration();
     return fsAccess.execute(hadoopUser, conf, executor);
     return fsAccess.execute(hadoopUser, conf, executor);
   }
   }
 
 
@@ -194,7 +194,7 @@ public class HttpFSServer {
   private FileSystem createFileSystem(Principal user, String doAs) throws IOException, FileSystemAccessException {
   private FileSystem createFileSystem(Principal user, String doAs) throws IOException, FileSystemAccessException {
     String hadoopUser = getEffectiveUser(user, doAs);
     String hadoopUser = getEffectiveUser(user, doAs);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
-    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getDefaultConfiguration();
+    Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getFileSystemConfiguration();
     FileSystem fs = fsAccess.createFileSystem(hadoopUser, conf);
     FileSystem fs = fsAccess.createFileSystem(hadoopUser, conf);
     FileSystemReleaseFilter.setFileSystem(fs);
     FileSystemReleaseFilter.setFileSystem(fs);
     return fs;
     return fs;

+ 16 - 9
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.http.server;
 package org.apache.hadoop.fs.http.server;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.servlet.ServerWebApp;
 import org.apache.hadoop.lib.servlet.ServerWebApp;
@@ -29,8 +30,9 @@ import java.io.IOException;
 
 
 /**
 /**
  * Bootstrap class that manages the initialization and destruction of the
  * Bootstrap class that manages the initialization and destruction of the
- * HttpFSServer server, it is a <code>javax.servlet.ServletContextListener</code>
- * implementation that is wired in HttpFSServer's WAR <code>WEB-INF/web.xml</code>.
+ * HttpFSServer server, it is a <code>javax.servlet.ServletContextListener
+ * </code> implementation that is wired in HttpFSServer's WAR
+ * <code>WEB-INF/web.xml</code>.
  * <p/>
  * <p/>
  * It provides acces to the server context via the singleton {@link #get}.
  * It provides acces to the server context via the singleton {@link #get}.
  * <p/>
  * <p/>
@@ -38,7 +40,8 @@ import java.io.IOException;
  * with <code>httpfs.</code>.
  * with <code>httpfs.</code>.
  */
  */
 public class HttpFSServerWebApp extends ServerWebApp {
 public class HttpFSServerWebApp extends ServerWebApp {
-  private static final Logger LOG = LoggerFactory.getLogger(HttpFSServerWebApp.class);
+  private static final Logger LOG =
+    LoggerFactory.getLogger(HttpFSServerWebApp.class);
 
 
   /**
   /**
    * Server name and prefix for all configuration properties.
    * Server name and prefix for all configuration properties.
@@ -67,8 +70,8 @@ public class HttpFSServerWebApp extends ServerWebApp {
   /**
   /**
    * Constructor used for testing purposes.
    * Constructor used for testing purposes.
    */
    */
-  protected HttpFSServerWebApp(String homeDir, String configDir, String logDir, String tempDir,
-                               Configuration config) {
+  protected HttpFSServerWebApp(String homeDir, String configDir, String logDir,
+                               String tempDir, Configuration config) {
     super(NAME, homeDir, configDir, logDir, tempDir, config);
     super(NAME, homeDir, configDir, logDir, tempDir, config);
   }
   }
 
 
@@ -80,9 +83,11 @@ public class HttpFSServerWebApp extends ServerWebApp {
   }
   }
 
 
   /**
   /**
-   * Initializes the HttpFSServer server, loads configuration and required services.
+   * Initializes the HttpFSServer server, loads configuration and required
+   * services.
    *
    *
-   * @throws ServerException thrown if HttpFSServer server could not be initialized.
+   * @throws ServerException thrown if HttpFSServer server could not be
+   * initialized.
    */
    */
   @Override
   @Override
   public void init() throws ServerException {
   public void init() throws ServerException {
@@ -93,7 +98,8 @@ public class HttpFSServerWebApp extends ServerWebApp {
     SERVER = this;
     SERVER = this;
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     LOG.info("Connects to Namenode [{}]",
     LOG.info("Connects to Namenode [{}]",
-             get().get(FileSystemAccess.class).getDefaultConfiguration().get("fs.default.name"));
+             get().get(FileSystemAccess.class).getFileSystemConfiguration().
+               get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
   }
   }
 
 
   /**
   /**
@@ -106,7 +112,8 @@ public class HttpFSServerWebApp extends ServerWebApp {
   }
   }
 
 
   /**
   /**
-   * Returns HttpFSServer server singleton, configuration and services are accessible through it.
+   * Returns HttpFSServer server singleton, configuration and services are
+   * accessible through it.
    *
    *
    * @return the HttpFSServer server singleton.
    * @return the HttpFSServer server singleton.
    */
    */

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java

@@ -37,6 +37,6 @@ public interface FileSystemAccess {
 
 
   public void releaseFileSystem(FileSystem fs) throws IOException;
   public void releaseFileSystem(FileSystem fs) throws IOException;
 
 
-  public Configuration getDefaultConfiguration();
+  public Configuration getFileSystemConfiguration();
 
 
 }
 }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java

@@ -26,12 +26,14 @@ public class FileSystemAccessException extends XException {
     H01("Service property [{0}] not defined"),
     H01("Service property [{0}] not defined"),
     H02("Kerberos initialization failed, {0}"),
     H02("Kerberos initialization failed, {0}"),
     H03("FileSystemExecutor error, {0}"),
     H03("FileSystemExecutor error, {0}"),
-    H04("JobClientExecutor error, {0}"),
+    H04("Invalid configuration, it has not be created by the FileSystemAccessService"),
     H05("[{0}] validation failed, {1}"),
     H05("[{0}] validation failed, {1}"),
     H06("Property [{0}] not defined in configuration object"),
     H06("Property [{0}] not defined in configuration object"),
     H07("[{0}] not healthy, {1}"),
     H07("[{0}] not healthy, {1}"),
-    H08(""),
-    H09("Invalid FileSystemAccess security mode [{0}]");
+    H08("{0}"),
+    H09("Invalid FileSystemAccess security mode [{0}]"),
+    H10("Hadoop config directory not found [{0}]"),
+    H11("Could not load Hadoop config files, {0}");
 
 
     private String template;
     private String template;
 
 

+ 52 - 35
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java

@@ -19,7 +19,9 @@
 package org.apache.hadoop.lib.service.hadoop;
 package org.apache.hadoop.lib.service.hadoop;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.service.FileSystemAccess;
@@ -32,6 +34,7 @@ import org.apache.hadoop.util.VersionInfo;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
@@ -54,9 +57,11 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
 
 
   public static final String NAME_NODE_WHITELIST = "name.node.whitelist";
   public static final String NAME_NODE_WHITELIST = "name.node.whitelist";
 
 
-  private static final String HADOOP_CONF_PREFIX = "conf:";
+  public static final String HADOOP_CONF_DIR = "config.dir";
 
 
-  private static final String NAME_NODE_PROPERTY = "fs.default.name";
+  private static final String[] HADOOP_CONF_FILES = {"core-site.xml", "hdfs-site.xml"};
+
+  private static final String FILE_SYSTEM_SERVICE_CREATED = "FileSystemAccessService.created";
 
 
   public FileSystemAccessService() {
   public FileSystemAccessService() {
     super(PREFIX);
     super(PREFIX);
@@ -102,26 +107,40 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
       throw new ServiceException(FileSystemAccessException.ERROR.H09, security);
       throw new ServiceException(FileSystemAccessException.ERROR.H09, security);
     }
     }
 
 
-    serviceHadoopConf = new Configuration(false);
-    for (Map.Entry entry : getServiceConfig()) {
-      String name = (String) entry.getKey();
-      if (name.startsWith(HADOOP_CONF_PREFIX)) {
-        name = name.substring(HADOOP_CONF_PREFIX.length());
-        String value = (String) entry.getValue();
-        serviceHadoopConf.set(name, value);
-
-      }
+    String hadoopConfDirProp = getServiceConfig().get(HADOOP_CONF_DIR, getServer().getConfigDir());
+    File hadoopConfDir = new File(hadoopConfDirProp).getAbsoluteFile();
+    if (hadoopConfDir == null) {
+      hadoopConfDir = new File(getServer().getConfigDir()).getAbsoluteFile();
+    }
+    if (!hadoopConfDir.exists()) {
+      throw new ServiceException(FileSystemAccessException.ERROR.H10, hadoopConfDir);
+    }
+    try {
+      serviceHadoopConf = loadHadoopConf(hadoopConfDir);
+    } catch (IOException ex) {
+      throw new ServiceException(FileSystemAccessException.ERROR.H11, ex.toString(), ex);
     }
     }
-    setRequiredServiceHadoopConf(serviceHadoopConf);
 
 
-    LOG.debug("FileSystemAccess default configuration:");
+    LOG.debug("FileSystemAccess FileSystem configuration:");
     for (Map.Entry entry : serviceHadoopConf) {
     for (Map.Entry entry : serviceHadoopConf) {
       LOG.debug("  {} = {}", entry.getKey(), entry.getValue());
       LOG.debug("  {} = {}", entry.getKey(), entry.getValue());
     }
     }
+    setRequiredServiceHadoopConf(serviceHadoopConf);
 
 
     nameNodeWhitelist = toLowerCase(getServiceConfig().getTrimmedStringCollection(NAME_NODE_WHITELIST));
     nameNodeWhitelist = toLowerCase(getServiceConfig().getTrimmedStringCollection(NAME_NODE_WHITELIST));
   }
   }
 
 
+  private Configuration loadHadoopConf(File dir) throws IOException {
+    Configuration hadoopConf = new Configuration(false);
+    for (String file : HADOOP_CONF_FILES) {
+      File f = new File(dir, file);
+      if (f.exists()) {
+        hadoopConf.addResource(new Path(f.getAbsolutePath()));
+      }
+    }
+    return hadoopConf;
+  }
+
   @Override
   @Override
   public void postInit() throws ServiceException {
   public void postInit() throws ServiceException {
     super.postInit();
     super.postInit();
@@ -166,17 +185,6 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     conf.set("fs.hdfs.impl.disable.cache", "true");
     conf.set("fs.hdfs.impl.disable.cache", "true");
   }
   }
 
 
-  protected Configuration createHadoopConf(Configuration conf) {
-    Configuration hadoopConf = new Configuration();
-    ConfigurationUtils.copy(serviceHadoopConf, hadoopConf);
-    ConfigurationUtils.copy(conf, hadoopConf);
-    return hadoopConf;
-  }
-
-  protected Configuration createNameNodeConf(Configuration conf) {
-    return createHadoopConf(conf);
-  }
-
   protected FileSystem createFileSystem(Configuration namenodeConf) throws IOException {
   protected FileSystem createFileSystem(Configuration namenodeConf) throws IOException {
     return FileSystem.get(namenodeConf);
     return FileSystem.get(namenodeConf);
   }
   }
@@ -202,16 +210,22 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     Check.notEmpty(user, "user");
     Check.notEmpty(user, "user");
     Check.notNull(conf, "conf");
     Check.notNull(conf, "conf");
     Check.notNull(executor, "executor");
     Check.notNull(executor, "executor");
-    if (conf.get(NAME_NODE_PROPERTY) == null || conf.getTrimmed(NAME_NODE_PROPERTY).length() == 0) {
-      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H06, NAME_NODE_PROPERTY);
+    if (!conf.getBoolean(FILE_SYSTEM_SERVICE_CREATED, false)) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H04);
+    }
+    if (conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY) == null ||
+        conf.getTrimmed(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY).length() == 0) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H06,
+                                          CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     }
     }
     try {
     try {
-      validateNamenode(new URI(conf.get(NAME_NODE_PROPERTY)).getAuthority());
+      validateNamenode(
+        new URI(conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)).
+          getAuthority());
       UserGroupInformation ugi = getUGI(user);
       UserGroupInformation ugi = getUGI(user);
       return ugi.doAs(new PrivilegedExceptionAction<T>() {
       return ugi.doAs(new PrivilegedExceptionAction<T>() {
         public T run() throws Exception {
         public T run() throws Exception {
-          Configuration namenodeConf = createNameNodeConf(conf);
-          FileSystem fs = createFileSystem(namenodeConf);
+          FileSystem fs = createFileSystem(conf);
           Instrumentation instrumentation = getServer().get(Instrumentation.class);
           Instrumentation instrumentation = getServer().get(Instrumentation.class);
           Instrumentation.Cron cron = instrumentation.createCron();
           Instrumentation.Cron cron = instrumentation.createCron();
           try {
           try {
@@ -236,13 +250,16 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     throws IOException, FileSystemAccessException {
     throws IOException, FileSystemAccessException {
     Check.notEmpty(user, "user");
     Check.notEmpty(user, "user");
     Check.notNull(conf, "conf");
     Check.notNull(conf, "conf");
+    if (!conf.getBoolean(FILE_SYSTEM_SERVICE_CREATED, false)) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H04);
+    }
     try {
     try {
-      validateNamenode(new URI(conf.get(NAME_NODE_PROPERTY)).getAuthority());
+      validateNamenode(
+        new URI(conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)).getAuthority());
       UserGroupInformation ugi = getUGI(user);
       UserGroupInformation ugi = getUGI(user);
       return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
       return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
         public FileSystem run() throws Exception {
         public FileSystem run() throws Exception {
-          Configuration namenodeConf = createNameNodeConf(conf);
-          return createFileSystem(namenodeConf);
+          return createFileSystem(conf);
         }
         }
       });
       });
     } catch (IOException ex) {
     } catch (IOException ex) {
@@ -267,11 +284,11 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     closeFileSystem(fs);
     closeFileSystem(fs);
   }
   }
 
 
-
   @Override
   @Override
-  public Configuration getDefaultConfiguration() {
-    Configuration conf = new Configuration(false);
+  public Configuration getFileSystemConfiguration() {
+    Configuration conf = new Configuration(true);
     ConfigurationUtils.copy(serviceHadoopConf, conf);
     ConfigurationUtils.copy(serviceHadoopConf, conf);
+    conf.setBoolean(FILE_SYSTEM_SERVICE_CREATED, true);
     return conf;
     return conf;
   }
   }
 
 

+ 0 - 31
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml

@@ -153,29 +153,6 @@
     </description>
     </description>
   </property>
   </property>
 
 
-  <!-- FileSystemAccess Namenode Configuration -->
-
-  <property>
-    <name>namenode.hostname</name>
-    <value>localhost</value>
-    <description>
-      The HDFS Namenode host the httpfs server connects to perform file
-      system operations.
-
-      This property is only used to resolve other properties within this
-      configuration file.
-    </description>
-  </property>
-
-  <property>
-    <name>httpfs.hadoop.conf:fs.default.name</name>
-    <value>hdfs://${namenode.hostname}:8020</value>
-    <description>
-      The HDFS Namenode URI the httpfs server connects to perform file
-      system operations.
-    </description>
-  </property>
-
   <!-- FileSystemAccess Namenode Security Configuration -->
   <!-- FileSystemAccess Namenode Security Configuration -->
 
 
   <property>
   <property>
@@ -206,12 +183,4 @@
     </description>
     </description>
   </property>
   </property>
 
 
-  <property>
-    <name>httpfs.hadoop.conf:dfs.namenode.kerberos.principal</name>
-    <value>hdfs/${namenode.hostname}@${kerberos.realm}</value>
-    <description>
-      The HDFS Namenode Kerberos principal.
-    </description>
-  </property>
-
 </configuration>
 </configuration>

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm

@@ -37,13 +37,13 @@ Hadoop HDFS over HTTP ${project.version} - Server Setup
 
 
 * Configure HttpFS
 * Configure HttpFS
 
 
-  Edit the <<<httpfs-${project.version}/conf/httpfs-site.xml>>> file and
-  set the <<<httpfs.fsAccess.conf:fs.default.name>>> property to the HDFS
-  Namenode URI. For example:
+  By default, HttpFS assumes that Hadoop configuration files
+  (<<<core-site.xml & hdfs-site.xml>>>) are in the HttpFS
+  configuration directory.
 
 
-+---+
-httpfs.fsAccess.conf:fs.default.name=hdfs://localhost:8021
-+---+
+  If this is not the case, add to the <<<httpfs-site.xml>>> file the
+  <<<httpfs.hadoop.config.dir>>> property set to the location
+  of the Hadoop configuration directory.
 
 
 * Configure Hadoop
 * Configure Hadoop
 
 
@@ -53,11 +53,11 @@ httpfs.fsAccess.conf:fs.default.name=hdfs://localhost:8021
 +---+
 +---+
   ...
   ...
   <property>
   <property>
-    <name>fsAccess.proxyuser.#HTTPFSUSER#.hosts</name>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.hosts</name>
     <value>httpfs-host.foo.com</value>
     <value>httpfs-host.foo.com</value>
   </property>
   </property>
   <property>
   <property>
-    <name>fsAccess.proxyuser.#HTTPFSUSER#.groups</name>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.groups</name>
     <value>*</value>
     <value>*</value>
   </property>
   </property>
   ...
   ...

+ 17 - 8
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.http.client;
 package org.apache.hadoop.fs.http.client;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -70,16 +71,24 @@ public class TestHttpFSFileSystem extends HFSTestCase {
     w.write("secret");
     w.write("secret");
     w.close();
     w.close();
 
 
-    String fsDefaultName = TestHdfsHelper.getHdfsConf().get("fs.default.name");
+    //HDFS configuration
+    String fsDefaultName = TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
-    conf.set("httpfs.hadoop.conf:fs.default.name", fsDefaultName);
-    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups", HadoopUsersConfTestHelper
-      .getHadoopProxyUserGroups());
-    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts", HadoopUsersConfTestHelper
-      .getHadoopProxyUserHosts());
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(new File(homeDir, "conf"), "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    //HTTPFS configuration
+    conf = new Configuration(false);
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups",
+             HadoopUsersConfTestHelper.getHadoopProxyUserGroups());
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
+             HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
-    File hoopSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
-    OutputStream os = new FileOutputStream(hoopSite);
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
     conf.writeXml(os);
     conf.writeXml(os);
     os.close();
     os.close();
 
 

+ 70 - 11
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -20,10 +20,12 @@ package org.apache.hadoop.fs.http.server;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.lib.service.security.DummyGroupMapping;
+import org.apache.hadoop.lib.server.Service;
+import org.apache.hadoop.lib.server.ServiceException;
+import org.apache.hadoop.lib.service.Groups;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 import org.apache.hadoop.test.TestDir;
 import org.apache.hadoop.test.TestDir;
@@ -40,12 +42,15 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
 import java.io.FileWriter;
+import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.io.Writer;
 import java.io.Writer;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.net.URL;
 import java.text.MessageFormat;
 import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.List;
 
 
 public class TestHttpFSServer extends HFSTestCase {
 public class TestHttpFSServer extends HFSTestCase {
 
 
@@ -54,12 +59,48 @@ public class TestHttpFSServer extends HFSTestCase {
   @TestJetty
   @TestJetty
   public void server() throws Exception {
   public void server() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration hoopConf = new Configuration(false);
-    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, hoopConf);
+
+    Configuration httpfsConf = new Configuration(false);
+    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
     server.init();
     server.init();
     server.destroy();
     server.destroy();
   }
   }
 
 
+  public static class MockGroups implements Service,Groups {
+
+    @Override
+    public void init(org.apache.hadoop.lib.server.Server server) throws ServiceException {
+    }
+
+    @Override
+    public void postInit() throws ServiceException {
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+    @Override
+    public Class[] getServiceDependencies() {
+      return new Class[0];
+    }
+
+    @Override
+    public Class getInterface() {
+      return Groups.class;
+    }
+
+    @Override
+    public void serverStatusChange(org.apache.hadoop.lib.server.Server.Status oldStatus,
+                                   org.apache.hadoop.lib.server.Server.Status newStatus) throws ServiceException {
+    }
+
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      return Arrays.asList(HadoopUsersConfTestHelper.getHadoopUserGroups(user));
+    }
+
+  }
   private void createHttpFSServer() throws Exception {
   private void createHttpFSServer() throws Exception {
     File homeDir = TestDirHelper.getTestDir();
     File homeDir = TestDirHelper.getTestDir();
     Assert.assertTrue(new File(homeDir, "conf").mkdir());
     Assert.assertTrue(new File(homeDir, "conf").mkdir());
@@ -72,13 +113,29 @@ public class TestHttpFSServer extends HFSTestCase {
     w.write("secret");
     w.write("secret");
     w.close();
     w.close();
 
 
-    String fsDefaultName = TestHdfsHelper.getHdfsConf().get("fs.default.name");
+    //HDFS configuration
+    File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
+    hadoopConfDir.mkdirs();
+    String fsDefaultName = TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
-    conf.set("httpfs.hadoop.conf:fs.default.name", fsDefaultName);
-    conf.set("httpfs.groups." + CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, DummyGroupMapping.class.getName());
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    //HTTPFS configuration
+    conf = new Configuration(false);
+    conf.set("httpfs.services.ext", MockGroups.class.getName());
+    conf.set("httpfs.admin.group", HadoopUsersConfTestHelper.
+      getHadoopUserGroups(HadoopUsersConfTestHelper.getHadoopUsers()[0])[0]);
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups",
+             HadoopUsersConfTestHelper.getHadoopProxyUserGroups());
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
+             HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
-    File hoopSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
-    OutputStream os = new FileOutputStream(hoopSite);
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
     conf.writeXml(os);
     conf.writeXml(os);
     os.close();
     os.close();
 
 
@@ -103,7 +160,8 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
 
 
     url = new URL(TestJettyHelper.getJettyURL(),
     url = new URL(TestJettyHelper.getJettyURL(),
-                  MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "root"));
+                  MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
+                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
     conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
@@ -112,7 +170,8 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertTrue(line.contains("\"counters\":{"));
     Assert.assertTrue(line.contains("\"counters\":{"));
 
 
     url = new URL(TestJettyHelper.getJettyURL(),
     url = new URL(TestJettyHelper.getJettyURL(),
-                  MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation", "root"));
+                  MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation",
+                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
     conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
   }
   }

+ 71 - 6
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.lib.service.hadoop;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.lib.server.Server;
 import org.apache.hadoop.lib.server.Server;
@@ -34,13 +35,32 @@ import org.apache.hadoop.test.TestException;
 import org.apache.hadoop.test.TestHdfs;
 import org.apache.hadoop.test.TestHdfs;
 import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
 public class TestFileSystemAccessService extends HFSTestCase {
 public class TestFileSystemAccessService extends HFSTestCase {
 
 
+  private void createHadoopConf(Configuration hadoopConf) throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    File hdfsSite = new File(dir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    hadoopConf.writeXml(os);
+    os.close();
+  }
+
+  @Before
+  public void createHadoopConf() throws Exception {
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set("foo", "FOO");
+    createHadoopConf(hadoopConf);
+  }
+
   @Test
   @Test
   @TestDir
   @TestDir
   public void simpleSecurity() throws Exception {
   public void simpleSecurity() throws Exception {
@@ -124,7 +144,7 @@ public class TestFileSystemAccessService extends HFSTestCase {
                                                           FileSystemAccessService.class.getName()));
                                                           FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.services", services);
-    conf.set("server.hadoop.conf:foo", "FOO");
+
     Server server = new Server("server", dir, dir, dir, dir, conf);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     server.init();
     FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class);
     FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class);
@@ -132,6 +152,32 @@ public class TestFileSystemAccessService extends HFSTestCase {
     server.destroy();
     server.destroy();
   }
   }
 
 
+  @Test
+  @TestDir
+  public void serviceHadoopConfCustomDir() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    String hadoopConfDir = new File(dir, "confx").getAbsolutePath();
+    new File(hadoopConfDir).mkdirs();
+    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
+                                                          FileSystemAccessService.class.getName()));
+    Configuration conf = new Configuration(false);
+    conf.set("server.services", services);
+    conf.set("server.hadoop.config.dir", hadoopConfDir);
+
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set("foo", "BAR");
+    hadoopConf.writeXml(os);
+    os.close();
+
+    Server server = new Server("server", dir, dir, dir, dir, conf);
+    server.init();
+    FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class);
+    Assert.assertEquals(fsAccess.serviceHadoopConf.get("foo"), "BAR");
+    server.destroy();
+  }
+
   @Test
   @Test
   @TestDir
   @TestDir
   public void inWhitelists() throws Exception {
   public void inWhitelists() throws Exception {
@@ -188,12 +234,17 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
                                                           FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     server.init();
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
-    FileSystem fs = hadoop.createFileSystem("u", TestHdfsHelper.getHdfsConf());
+    FileSystem fs = hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration());
     Assert.assertNotNull(fs);
     Assert.assertNotNull(fs);
     fs.mkdirs(new Path("/tmp/foo"));
     fs.mkdirs(new Path("/tmp/foo"));
     hadoop.releaseFileSystem(fs);
     hadoop.releaseFileSystem(fs);
@@ -214,6 +265,11 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
                                                           FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     Server server = new Server("server", dir, dir, dir, dir, conf);
@@ -222,7 +278,7 @@ public class TestFileSystemAccessService extends HFSTestCase {
 
 
     final FileSystem fsa[] = new FileSystem[1];
     final FileSystem fsa[] = new FileSystem[1];
 
 
-    hadoop.execute("u", TestHdfsHelper.getHdfsConf(), new FileSystemAccess.FileSystemExecutor<Void>() {
+    hadoop.execute("u", hadoop.getFileSystemConfiguration(), new FileSystemAccess.FileSystemExecutor<Void>() {
       @Override
       @Override
       public Void execute(FileSystem fs) throws IOException {
       public Void execute(FileSystem fs) throws IOException {
         fs.mkdirs(new Path("/tmp/foo"));
         fs.mkdirs(new Path("/tmp/foo"));
@@ -248,14 +304,18 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
                                                           FileSystemAccessService.class.getName()));
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     server.init();
     FileSystemAccess fsAccess = server.get(FileSystemAccess.class);
     FileSystemAccess fsAccess = server.get(FileSystemAccess.class);
 
 
-    Configuration hdfsConf = TestHdfsHelper.getHdfsConf();
-    hdfsConf.set("fs.default.name", "");
+    Configuration hdfsConf = fsAccess.getFileSystemConfiguration();
+    hdfsConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "");
     fsAccess.execute("u", hdfsConf, new FileSystemAccess.FileSystemExecutor<Void>() {
     fsAccess.execute("u", hdfsConf, new FileSystemAccess.FileSystemExecutor<Void>() {
       @Override
       @Override
       public Void execute(FileSystem fs) throws IOException {
       public Void execute(FileSystem fs) throws IOException {
@@ -271,6 +331,11 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
                                                           FileSystemAccessService.class.getName()));
                                                           FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     Server server = new Server("server", dir, dir, dir, dir, conf);
@@ -279,7 +344,7 @@ public class TestFileSystemAccessService extends HFSTestCase {
 
 
     final FileSystem fsa[] = new FileSystem[1];
     final FileSystem fsa[] = new FileSystem[1];
     try {
     try {
-      hadoop.execute("u", TestHdfsHelper.getHdfsConf(), new FileSystemAccess.FileSystemExecutor<Void>() {
+      hadoop.execute("u", hadoop.getFileSystemConfiguration(), new FileSystemAccess.FileSystemExecutor<Void>() {
         @Override
         @Override
         public Void execute(FileSystem fs) throws IOException {
         public Void execute(FileSystem fs) throws IOException {
           fsa[0] = fs;
           fsa[0] = fs;

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

@@ -145,7 +145,12 @@ public class HadoopUsersConfTestHelper {
    */
    */
   public static String[] getHadoopUserGroups(String user) {
   public static String[] getHadoopUserGroups(String user) {
     if (getHadoopUsers() == DEFAULT_USERS) {
     if (getHadoopUsers() == DEFAULT_USERS) {
-      return DEFAULT_USERS_GROUP;
+      for (String defaultUser : DEFAULT_USERS) {
+        if (defaultUser.equals(user)) {
+          return DEFAULT_USERS_GROUP;
+        }
+      }
+      return new String[0];
     } else {
     } else {
       String groups = System.getProperty(HADOOP_USER_PREFIX + user);
       String groups = System.getProperty(HADOOP_USER_PREFIX + user);
       return (groups != null) ? groups.split(",") : new String[0];
       return (groups != null) ? groups.split(",") : new String[0];

+ 48 - 2
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -62,6 +62,12 @@ Trunk (unreleased changes)
     HDFS-3178. Add states and state handler for journal synchronization in
     HDFS-3178. Add states and state handler for journal synchronization in
     JournalService.  (szetszwo)
     JournalService.  (szetszwo)
 
 
+    HDFS-3273. Refactor BackupImage and FSEditLog, and rename
+    JournalListener.rollLogs(..) to startLogSegment(..).  (szetszwo)
+
+    HDFS-3292. Remove the deprecated DiskStatus, getDiskStatus(), getRawCapacity() and
+    getRawUsed() from DistributedFileSystem.  (Arpit Gupta via szetszwo)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-2834. Add a ByteBuffer-based read API to DFSInputStream.
     HDFS-2834. Add a ByteBuffer-based read API to DFSInputStream.
@@ -91,8 +97,6 @@ Trunk (unreleased changes)
     HDFS-2373. Commands using WebHDFS and hftp print unnecessary debug 
     HDFS-2373. Commands using WebHDFS and hftp print unnecessary debug 
     info on the console with security enabled. (Arpit Gupta via suresh)
     info on the console with security enabled. (Arpit Gupta via suresh)
 
 
-    HDFS-2765. TestNameEditsConfigs is incorrectly swallowing IOE. (atm)
-
     HDFS-2776. Missing interface annotation on JournalSet. 
     HDFS-2776. Missing interface annotation on JournalSet. 
     (Brandon Li via jitendra)
     (Brandon Li via jitendra)
 
 
@@ -367,6 +371,25 @@ Release 2.0.0 - UNRELEASED
 
 
     HDFS-3249. Use ToolRunner.confirmPrompt in NameNode (todd)
     HDFS-3249. Use ToolRunner.confirmPrompt in NameNode (todd)
 
 
+    HDFS-3179.  Improve the exception message thrown by DataStreamer when 
+    it failed to add a datanode.  (szetszwo)
+
+    HDFS-2983. Relax the build version check to permit rolling upgrades within
+    a release. (atm)
+
+    HDFS-3259. NameNode#initializeSharedEdits should populate shared edits dir
+    with edit log segments. (atm)
+
+    HDFS-2708. Stats for the # of blocks per DN. (atm)
+
+    HDFS-3279. Move the FSEditLog constructor with @VisibleForTesting to
+    TestEditLog.  (Arpit Gupta via szetszwo)
+
+    HDFS-3294. Fix code indentation in NamenodeWebHdfsMethods and
+    DatanodeWebHdfsMethods.  (szetszwo)
+
+    HDFS-3263. HttpFS should read HDFS config from Hadoop site.xml files (tucu)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -492,6 +515,26 @@ Release 2.0.0 - UNRELEASED
 
 
     HDFS-2696. Fix the fuse-fds build. (Bruno Mahé via eli)
     HDFS-2696. Fix the fuse-fds build. (Bruno Mahé via eli)
 
 
+    HDFS-3260. TestDatanodeRegistration should set minimum DN version in
+    addition to minimum NN version. (atm)
+
+    HDFS-3255. HA DFS returns wrong token service (Daryn Sharp via todd)
+
+    HDFS-3256. HDFS considers blocks under-replicated if topology script is
+    configured with only 1 rack. (atm)
+
+    HDFS-2799. Trim fs.checkpoint.dir values. (Amith D K via eli)
+
+    HDFS-2765. TestNameEditsConfigs is incorrectly swallowing IOE. (atm)
+
+    HDFS-3268. FileContext API mishandles token service and incompatible with
+    HA (Daryn Sharp via todd)
+
+    HDFS-3284. bootstrapStandby fails in secure cluster (todd)
+
+    HDFS-3165. HDFS Balancer scripts are refering to wrong path of
+    hadoop-daemon.sh (Amith D K via eli)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
@@ -825,6 +868,9 @@ Release 0.23.3 - UNRELEASED
     HDFS-3176. Use MD5MD5CRC32FileChecksum.readFields() in JsonUtil .  (Kihwal
     HDFS-3176. Use MD5MD5CRC32FileChecksum.readFields() in JsonUtil .  (Kihwal
     Lee via szetszwo)
     Lee via szetszwo)
 
 
+    HDFS-2652. Add support for host-based delegation tokens.  (Daryn Sharp via
+    szetszwo)
+
 Release 0.23.2 - UNRELEASED
 Release 0.23.2 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-balancer.sh

@@ -24,4 +24,4 @@ HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
 
 
 # Start balancer daemon.
 # Start balancer daemon.
 
 
-"$HADOOP_PREFIX"/bin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start balancer $@
+"$HADOOP_PREFIX"/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start balancer $@

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-balancer.sh

@@ -25,4 +25,4 @@ HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
 # Stop balancer daemon.
 # Stop balancer daemon.
 # Run this on the machine where the balancer is running
 # Run this on the machine where the balancer is running
 
 
-"$HADOOP_PREFIX"/bin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs stop balancer
+"$HADOOP_PREFIX"/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs stop balancer

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.fs;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -391,11 +390,15 @@ public class Hdfs extends AbstractFileSystem {
     return new Path(dfs.getLinkTarget(getUriPath(p)));
     return new Path(dfs.getLinkTarget(getUriPath(p)));
   }
   }
   
   
+  @Override
+  public String getCanonicalServiceName() {
+    return dfs.getCanonicalServiceName();
+  }
+
   @Override //AbstractFileSystem
   @Override //AbstractFileSystem
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
     Token<DelegationTokenIdentifier> result = dfs
     Token<DelegationTokenIdentifier> result = dfs
         .getDelegationToken(renewer == null ? null : new Text(renewer));
         .getDelegationToken(renewer == null ? null : new Text(renewer));
-    result.setService(new Text(this.getCanonicalServiceName()));
     List<Token<?>> tokenList = new ArrayList<Token<?>>();
     List<Token<?>> tokenList = new ArrayList<Token<?>>();
     tokenList.add(result);
     tokenList.add(result);
     return tokenList;
     return tokenList;

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -637,6 +637,16 @@ public class DFSClient implements java.io.Closeable {
     return serverDefaults;
     return serverDefaults;
   }
   }
   
   
+  /**
+   * Get a canonical token service name for this client's tokens.  Null should
+   * be returned if the client is not using tokens.
+   * @return the token service for the client
+   */
+  @InterfaceAudience.LimitedPrivate( { "HDFS" }) 
+  public String getCanonicalServiceName() {
+    return (dtService != null) ? dtService.toString() : null;
+  }
+  
   /**
   /**
    * @see ClientProtocol#getDelegationToken(Text)
    * @see ClientProtocol#getDelegationToken(Text)
    */
    */

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

@@ -146,6 +146,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
   public static final int     DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
   public static final String  DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
   public static final String  DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
   public static final int     DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
   public static final int     DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
+  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY = "dfs.namenode.min.supported.datanode.version";
+  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "3.0.0";
 
 
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
@@ -262,6 +264,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_IPC_ADDRESS_KEY = "dfs.datanode.ipc.address";
   public static final String  DFS_DATANODE_IPC_ADDRESS_KEY = "dfs.datanode.ipc.address";
   public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 50020;
   public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 50020;
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0" + DFS_DATANODE_IPC_DEFAULT_PORT;
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0" + DFS_DATANODE_IPC_DEFAULT_PORT;
+  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
+  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "3.0.0";
 
 
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;

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

@@ -128,7 +128,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
   private volatile boolean appendChunk = false;   // appending to existing partial block
   private volatile boolean appendChunk = false;   // appending to existing partial block
   private long initialFileSize = 0; // at time of file open
   private long initialFileSize = 0; // at time of file open
   private Progressable progress;
   private Progressable progress;
-  private short blockReplication; // replication factor of file
+  private final short blockReplication; // replication factor of file
   
   
   private class Packet {
   private class Packet {
     long    seqno;               // sequencenumber of buffer in block
     long    seqno;               // sequencenumber of buffer in block
@@ -775,9 +775,13 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private int findNewDatanode(final DatanodeInfo[] original
     private int findNewDatanode(final DatanodeInfo[] original
         ) throws IOException {
         ) throws IOException {
       if (nodes.length != original.length + 1) {
       if (nodes.length != original.length + 1) {
-        throw new IOException("Failed to add a datanode:"
-            + " nodes.length != original.length + 1, nodes="
-            + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
+        throw new IOException("Failed to add a datanode.  "
+            + "User may turn off this feature by setting "
+            + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
+            + " in configuration, where the current policy is "
+            + dfsClient.dtpReplaceDatanodeOnFailure
+            + ".  (Nodes: current=" + Arrays.asList(nodes)
+            + ", original=" + Arrays.asList(original) + ")");
       }
       }
       for(int i = 0; i < nodes.length; i++) {
       for(int i = 0; i < nodes.length; i++) {
         int j = 0;
         int j = 0;

+ 1 - 49
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -499,56 +499,12 @@ public class DistributedFileSystem extends FileSystem {
     return dfs;
     return dfs;
   }        
   }        
   
   
-  /** @deprecated Use {@link org.apache.hadoop.fs.FsStatus} instead */
-  @InterfaceAudience.Private
-  @Deprecated
-  public static class DiskStatus extends FsStatus {
-    public DiskStatus(FsStatus stats) {
-      super(stats.getCapacity(), stats.getUsed(), stats.getRemaining());
-    }
-
-    public DiskStatus(long capacity, long dfsUsed, long remaining) {
-      super(capacity, dfsUsed, remaining);
-    }
-
-    public long getDfsUsed() {
-      return super.getUsed();
-    }
-  }
-  
   @Override
   @Override
   public FsStatus getStatus(Path p) throws IOException {
   public FsStatus getStatus(Path p) throws IOException {
     statistics.incrementReadOps(1);
     statistics.incrementReadOps(1);
     return dfs.getDiskStatus();
     return dfs.getDiskStatus();
   }
   }
 
 
-  /** Return the disk usage of the filesystem, including total capacity,
-   * used space, and remaining space 
-   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
-   * instead */
-   @Deprecated
-  public DiskStatus getDiskStatus() throws IOException {
-    return new DiskStatus(dfs.getDiskStatus());
-  }
-  
-  /** Return the total raw capacity of the filesystem, disregarding
-   * replication.
-   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
-   * instead */
-   @Deprecated
-  public long getRawCapacity() throws IOException{
-    return dfs.getDiskStatus().getCapacity();
-  }
-
-  /** Return the total raw used space in the filesystem, disregarding
-   * replication.
-   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
-   * instead */
-   @Deprecated
-  public long getRawUsed() throws IOException{
-    return dfs.getDiskStatus().getUsed();
-  }
-   
   /**
   /**
    * Returns count of blocks with no good replicas left. Normally should be
    * Returns count of blocks with no good replicas left. Normally should be
    * zero.
    * zero.
@@ -848,11 +804,7 @@ public class DistributedFileSystem extends FileSystem {
    */
    */
   @Override
   @Override
   public String getCanonicalServiceName() {
   public String getCanonicalServiceName() {
-    if (HAUtil.isLogicalUri(getConf(), getUri())) {
-      return getUri().getHost();
-    } else {
-      return super.getCanonicalServiceName();
-    }
+    return dfs.getCanonicalServiceName();
   }
   }
 
 
   /**
   /**

+ 19 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java

@@ -52,6 +52,9 @@ public class HAUtil {
   private static final Log LOG = 
   private static final Log LOG = 
     LogFactory.getLog(HAUtil.class);
     LogFactory.getLog(HAUtil.class);
   
   
+  private static final DelegationTokenSelector tokenSelector =
+      new DelegationTokenSelector();
+
   private HAUtil() { /* Hidden constructor */ }
   private HAUtil() { /* Hidden constructor */ }
 
 
   /**
   /**
@@ -241,25 +244,28 @@ public class HAUtil {
    * one is found, clone it to also represent the underlying namenode address.
    * one is found, clone it to also represent the underlying namenode address.
    * @param ugi the UGI to modify
    * @param ugi the UGI to modify
    * @param haUri the logical URI for the cluster
    * @param haUri the logical URI for the cluster
-   * @param singleNNAddr one of the NNs in the cluster to which the token
+   * @param nnAddrs collection of NNs in the cluster to which the token
    * applies
    * applies
    */
    */
   public static void cloneDelegationTokenForLogicalUri(
   public static void cloneDelegationTokenForLogicalUri(
       UserGroupInformation ugi, URI haUri,
       UserGroupInformation ugi, URI haUri,
-      InetSocketAddress singleNNAddr) {
-    Text haService = buildTokenServiceForLogicalUri(haUri);
+      Collection<InetSocketAddress> nnAddrs) {
+    Text haService = HAUtil.buildTokenServiceForLogicalUri(haUri);
     Token<DelegationTokenIdentifier> haToken =
     Token<DelegationTokenIdentifier> haToken =
-        DelegationTokenSelector.selectHdfsDelegationToken(haService, ugi);
-    if (haToken == null) {
-      // no token
-      return;
+        tokenSelector.selectToken(haService, ugi.getTokens());
+    if (haToken != null) {
+      for (InetSocketAddress singleNNAddr : nnAddrs) {
+        Token<DelegationTokenIdentifier> specificToken =
+            new Token<DelegationTokenIdentifier>(haToken);
+        SecurityUtil.setTokenService(specificToken, singleNNAddr);
+        ugi.addToken(specificToken);
+        LOG.debug("Mapped HA service delegation token for logical URI " +
+            haUri + " to namenode " + singleNNAddr);
+      }
+    } else {
+      LOG.debug("No HA service delegation token found for logical URI " +
+          haUri);
     }
     }
-    Token<DelegationTokenIdentifier> specificToken =
-        new Token<DelegationTokenIdentifier>(haToken);
-    specificToken.setService(SecurityUtil.buildTokenService(singleNNAddr));
-    ugi.addToken(specificToken);
-    LOG.debug("Mapped HA service delegation token for logical URI " +
-        haUri + " to namenode " + singleNNAddr);
   }
   }
 
 
   /**
   /**

+ 17 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -30,6 +30,7 @@ import java.security.PrivilegedExceptionAction;
 import java.text.ParseException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.TimeZone;
 import java.util.TimeZone;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -48,7 +49,6 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenRenewer;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenRenewer;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.hdfs.web.URLUtils;
 import org.apache.hadoop.hdfs.web.URLUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -168,10 +168,7 @@ public class HftpFileSystem extends FileSystem
 
 
   protected void initDelegationToken() throws IOException {
   protected void initDelegationToken() throws IOException {
     // look for hftp token, then try hdfs
     // look for hftp token, then try hdfs
-    Token<?> token = selectHftpDelegationToken();
-    if (token == null) {
-      token = selectHdfsDelegationToken();
-    }  
+    Token<?> token = selectDelegationToken();
 
 
     // if we don't already have a token, go get one over https
     // if we don't already have a token, go get one over https
     boolean createdToken = false;
     boolean createdToken = false;
@@ -192,14 +189,8 @@ public class HftpFileSystem extends FileSystem
     }
     }
   }
   }
 
 
-  protected Token<DelegationTokenIdentifier> selectHftpDelegationToken() {
-    Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr);
-    return hftpTokenSelector.selectToken(serviceName, ugi.getTokens());
-  }
-
-  protected Token<DelegationTokenIdentifier> selectHdfsDelegationToken() {
-    return  DelegationTokenSelector.selectHdfsDelegationToken(
-        nnAddr, ugi, getConf());
+  protected Token<DelegationTokenIdentifier> selectDelegationToken() {
+  	return hftpTokenSelector.selectToken(getUri(), ugi.getTokens(), getConf());
   }
   }
   
   
 
 
@@ -699,9 +690,22 @@ public class HftpFileSystem extends FileSystem
   
   
   private static class HftpDelegationTokenSelector
   private static class HftpDelegationTokenSelector
   extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
   extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+    private static final DelegationTokenSelector hdfsTokenSelector =
+        new DelegationTokenSelector();
 
 
     public HftpDelegationTokenSelector() {
     public HftpDelegationTokenSelector() {
       super(TOKEN_KIND);
       super(TOKEN_KIND);
     }
     }
+    
+    Token<DelegationTokenIdentifier> selectToken(URI nnUri,
+        Collection<Token<?>> tokens, Configuration conf) {
+      Token<DelegationTokenIdentifier> token =
+          selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
+      if (token == null) {
+        // try to get a HDFS token
+        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf); 
+      }
+      return token;
+    }
   }
   }
 }
 }

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -386,7 +386,7 @@ public class PBHelper {
     StorageInfoProto storage = info.getStorageInfo();
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion(),
         info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion(),
-        info.getBuildVersion());
+        info.getBuildVersion(), info.getSoftwareVersion());
   }
   }
 
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -612,13 +612,14 @@ public class PBHelper {
         .newBuilder();
         .newBuilder();
     return builder.setDatanodeID(PBHelper.convert((DatanodeID) registration))
     return builder.setDatanodeID(PBHelper.convert((DatanodeID) registration))
         .setStorageInfo(PBHelper.convert(registration.getStorageInfo()))
         .setStorageInfo(PBHelper.convert(registration.getStorageInfo()))
-        .setKeys(PBHelper.convert(registration.getExportedKeys())).build();
+        .setKeys(PBHelper.convert(registration.getExportedKeys()))
+        .setSoftwareVersion(registration.getSoftwareVersion()).build();
   }
   }
 
 
   public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
   public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
     return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
     return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
         PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
         PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
-            .getKeys()));
+            .getKeys()), proto.getSoftwareVersion());
   }
   }
 
 
   public static DatanodeCommand convert(DatanodeCommandProto proto) {
   public static DatanodeCommand convert(DatanodeCommandProto proto) {
@@ -894,7 +895,8 @@ public class PBHelper {
         .setBlockPoolID(info.getBlockPoolID())
         .setBlockPoolID(info.getBlockPoolID())
         .setBuildVersion(info.getBuildVersion())
         .setBuildVersion(info.getBuildVersion())
         .setDistUpgradeVersion(info.getDistributedUpgradeVersion())
         .setDistUpgradeVersion(info.getDistributedUpgradeVersion())
-        .setStorageInfo(PBHelper.convert((StorageInfo)info)).build();
+        .setStorageInfo(PBHelper.convert((StorageInfo)info))
+        .setSoftwareVersion(info.getSoftwareVersion()).build();
   }
   }
   
   
   // Located Block Arrays and Lists
   // Located Block Arrays and Lists

+ 20 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java

@@ -17,7 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.security.token.delegation;
 package org.apache.hadoop.hdfs.security.token.delegation;
 
 
-import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collection;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -25,7 +26,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 
 
@@ -37,32 +37,35 @@ public class DelegationTokenSelector
     extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
     extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
   public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
   public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
 
 
-  private static final DelegationTokenSelector INSTANCE = new DelegationTokenSelector();
-
-  /** Select the delegation token for hdfs from the ugi. */
-  public static Token<DelegationTokenIdentifier> selectHdfsDelegationToken(
-      final InetSocketAddress nnAddr, final UserGroupInformation ugi,
+  /**
+   * Select the delegation token for hdfs.  The port will be rewritten to
+   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port. 
+   * This method should only be called by non-hdfs filesystems that do not
+   * use the rpc port to acquire tokens.  Ex. webhdfs, hftp 
+   * @param nnUri of the remote namenode
+   * @param tokens as a collection
+   * @param conf hadoop configuration
+   * @return Token
+   */
+  public Token<DelegationTokenIdentifier> selectToken(
+      final URI nnUri, Collection<Token<?>> tokens,
       final Configuration conf) {
       final Configuration conf) {
     // this guesses the remote cluster's rpc service port.
     // this guesses the remote cluster's rpc service port.
     // the current token design assumes it's the same as the local cluster's
     // the current token design assumes it's the same as the local cluster's
     // rpc port unless a config key is set.  there should be a way to automatic
     // rpc port unless a config key is set.  there should be a way to automatic
     // and correctly determine the value
     // and correctly determine the value
-    final String key = SERVICE_NAME_KEY + SecurityUtil.buildTokenService(nnAddr);
-    final String nnServiceName = conf.get(key);
+    Text serviceName = SecurityUtil.buildTokenService(nnUri);
+    final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
     
     
     int nnRpcPort = NameNode.DEFAULT_PORT;
     int nnRpcPort = NameNode.DEFAULT_PORT;
     if (nnServiceName != null) {
     if (nnServiceName != null) {
       nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
       nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
     }
     }
+    // use original hostname from the uri to avoid unintentional host resolving
+    serviceName = SecurityUtil.buildTokenService(
+    		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
     
     
-    final Text serviceName = SecurityUtil.buildTokenService(
-        new InetSocketAddress(nnAddr.getHostName(), nnRpcPort));
-    return INSTANCE.selectToken(serviceName, ugi.getTokens());
-  }
-  
-  public static Token<DelegationTokenIdentifier> selectHdfsDelegationToken(
-      Text serviceName, UserGroupInformation ugi) {
-    return INSTANCE.selectToken(serviceName, ugi.getTokens());
+    return selectToken(serviceName, tokens);
   }
   }
 
 
   public DelegationTokenSelector() {
   public DelegationTokenSelector() {

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -247,8 +247,7 @@ public class BlockManager {
 
 
     this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
     this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
                                              DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
                                              DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
-    this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null ? false
-                                                                             : true;
+    this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) != null;
     
     
     this.replicationRecheckInterval = 
     this.replicationRecheckInterval = 
       conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 
       conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 
@@ -2829,7 +2828,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       DatanodeDescriptor cur = it.next();
       DatanodeDescriptor cur = it.next();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1) {
+          if (numExpectedReplicas == 1 ||
+              (numExpectedReplicas > 1 &&
+                  !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             enoughRacks = true;
             break;
             break;
           }
           }

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -71,6 +71,7 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.net.InetAddresses;
 import com.google.common.net.InetAddresses;
 
 
 /**
 /**
@@ -126,6 +127,12 @@ public class DatanodeManager {
   /** Ask Datanode only up to this many blocks to delete. */
   /** Ask Datanode only up to this many blocks to delete. */
   final int blockInvalidateLimit;
   final int blockInvalidateLimit;
   
   
+  /**
+   * Whether or not this cluster has ever consisted of more than 1 rack,
+   * according to the NetworkTopology.
+   */
+  private boolean hasClusterEverBeenMultiRack = false;
+  
   DatanodeManager(final BlockManager blockManager,
   DatanodeManager(final BlockManager blockManager,
       final Namesystem namesystem, final Configuration conf
       final Namesystem namesystem, final Configuration conf
       ) throws IOException {
       ) throws IOException {
@@ -331,6 +338,7 @@ public class DatanodeManager {
 
 
     host2DatanodeMap.add(node);
     host2DatanodeMap.add(node);
     networktopology.add(node);
     networktopology.add(node);
+    checkIfClusterIsNowMultiRack(node);
 
 
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(getClass().getSimpleName() + ".addDatanode: "
       LOG.debug(getClass().getSimpleName() + ".addDatanode: "
@@ -768,6 +776,42 @@ public class DatanodeManager {
     }
     }
   }
   }
 
 
+  /**
+   * @return true if this cluster has ever consisted of multiple racks, even if
+   *         it is not now a multi-rack cluster.
+   */
+  boolean hasClusterEverBeenMultiRack() {
+    return hasClusterEverBeenMultiRack;
+  }
+
+  /**
+   * Check if the cluster now consists of multiple racks. If it does, and this
+   * is the first time it's consisted of multiple racks, then process blocks
+   * that may now be misreplicated.
+   * 
+   * @param node DN which caused cluster to become multi-rack. Used for logging.
+   */
+  @VisibleForTesting
+  void checkIfClusterIsNowMultiRack(DatanodeDescriptor node) {
+    if (!hasClusterEverBeenMultiRack && networktopology.getNumOfRacks() > 1) {
+      String message = "DN " + node + " joining cluster has expanded a formerly " +
+          "single-rack cluster to be multi-rack. ";
+      if (namesystem.isPopulatingReplQueues()) {
+        message += "Re-checking all blocks for replication, since they should " +
+            "now be replicated cross-rack";
+        LOG.info(message);
+      } else {
+        message += "Not checking for mis-replicated blocks because this NN is " +
+            "not yet processing repl queues.";
+        LOG.debug(message);
+      }
+      hasClusterEverBeenMultiRack = true;
+      if (namesystem.isPopulatingReplQueues()) {
+        blockManager.processMisReplicatedBlocks();
+      }
+    }
+  }
+
   /**
   /**
    * Parse a DatanodeID from a hosts file entry
    * Parse a DatanodeID from a hosts file entry
    * @param hostLine of form [hostname|ip][:port]?
    * @param hostLine of form [hostname|ip][:port]?

+ 15 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java

@@ -32,7 +32,19 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class IncorrectVersionException extends IOException {
 public class IncorrectVersionException extends IOException {
   private static final long serialVersionUID = 1L;
   private static final long serialVersionUID = 1L;
+  
+  public IncorrectVersionException(String message) {
+    super(message);
+  }
 
 
+  public IncorrectVersionException(String minimumVersion, String reportedVersion,
+      String remoteDaemon, String thisDaemon) {
+    this("The reported " + remoteDaemon + " version is too low to communicate" +
+        " with this " + thisDaemon + ". " + remoteDaemon + " version: '" +
+        reportedVersion + "' Minimum " + remoteDaemon + " version: '" +
+        minimumVersion + "'");
+  }
+  
   public IncorrectVersionException(int versionReported, String ofWhat) {
   public IncorrectVersionException(int versionReported, String ofWhat) {
     this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION);
     this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION);
   }
   }
@@ -40,16 +52,9 @@ public class IncorrectVersionException extends IOException {
   public IncorrectVersionException(int versionReported,
   public IncorrectVersionException(int versionReported,
                                    String ofWhat,
                                    String ofWhat,
                                    int versionExpected) {
                                    int versionExpected) {
-    super("Unexpected version " 
-          + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
-          + versionReported + ". Expecting = " + versionExpected + ".");
+    this("Unexpected version " 
+        + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
+        + versionReported + ". Expecting = " + versionExpected + ".");
   }
   }
 
 
-  public IncorrectVersionException(String versionReported,
-                                   String ofWhat,
-                                   String versionExpected) {
-    super("Unexpected version " 
-          + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
-          + versionReported + ". Expecting = " + versionExpected + ".");
-  }
 }
 }

+ 16 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
-import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -52,6 +51,8 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.util.VersionUtil;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
@@ -178,17 +179,23 @@ class BPServiceActor implements Runnable {
   private void checkNNVersion(NamespaceInfo nsInfo)
   private void checkNNVersion(NamespaceInfo nsInfo)
       throws IncorrectVersionException {
       throws IncorrectVersionException {
     // build and layout versions should match
     // build and layout versions should match
-    String nsBuildVer = nsInfo.getBuildVersion();
-    String stBuildVer = Storage.getBuildVersion();
-    if (!nsBuildVer.equals(stBuildVer)) {
-      LOG.warn("Data-node and name-node Build versions must be the same. " +
-        "Namenode build version: " + nsBuildVer + "Datanode " +
-        "build version: " + stBuildVer);
-      throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
+    String nnVersion = nsInfo.getSoftwareVersion();
+    String minimumNameNodeVersion = dnConf.getMinimumNameNodeVersion();
+    if (VersionUtil.compareVersions(nnVersion, minimumNameNodeVersion) < 0) {
+      IncorrectVersionException ive = new IncorrectVersionException(
+          minimumNameNodeVersion, nnVersion, "NameNode", "DataNode");
+      LOG.warn(ive.getMessage());
+      throw ive;
+    }
+    String dnVersion = VersionInfo.getVersion();
+    if (!nnVersion.equals(dnVersion)) {
+      LOG.info("Reported NameNode version '" + nnVersion + "' does not match " +
+          "DataNode version '" + dnVersion + "' but is within acceptable " +
+          "limits. Note: This is normal during a rolling upgrade.");
     }
     }
 
 
     if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
     if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
-      LOG.warn("Data-node and name-node layout versions must be the same." +
+      LOG.warn("DataNode and NameNode layout versions must be the same." +
         " Expected: "+ HdfsConstants.LAYOUT_VERSION +
         " Expected: "+ HdfsConstants.LAYOUT_VERSION +
         " actual "+ nsInfo.getLayoutVersion());
         " actual "+ nsInfo.getLayoutVersion());
       throw new IncorrectVersionException(
       throw new IncorrectVersionException(

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java

@@ -31,6 +31,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOW
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -58,6 +60,8 @@ class DNConf {
   final long deleteReportInterval;
   final long deleteReportInterval;
   final long initialBlockReportDelay;
   final long initialBlockReportDelay;
   final int writePacketSize;
   final int writePacketSize;
+  
+  final String minimumNameNodeVersion;
 
 
   public DNConf(Configuration conf) {
   public DNConf(Configuration conf) {
     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
@@ -111,5 +115,12 @@ class DNConf {
     this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
     this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
         DFS_DATANODE_SYNCONCLOSE_DEFAULT);
         DFS_DATANODE_SYNCONCLOSE_DEFAULT);
 
 
+    this.minimumNameNodeVersion = conf.get(DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY,
+        DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT);
+  }
+  
+  // We get minimumNameNodeVersion via a method so it can be mocked out in tests.
+  String getMinimumNameNodeVersion() {
+    return this.minimumNameNodeVersion;
   }
   }
 }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -673,6 +673,7 @@ public class DataNode extends Configured
     bpRegistration.setIpcPort(getIpcPort());
     bpRegistration.setIpcPort(getIpcPort());
     bpRegistration.setHostName(hostName);
     bpRegistration.setHostName(hostName);
     bpRegistration.setStorageID(getStorageId());
     bpRegistration.setStorageID(getStorageId());
+    bpRegistration.setSoftwareVersion(VersionInfo.getVersion());
 
 
     StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
     StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
     if (storageInfo == null) {
     if (storageInfo == null) {

+ 47 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java

@@ -178,8 +178,25 @@ public class DatanodeWebHdfsMethods {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
       public Response run() throws IOException, URISyntaxException {
       public Response run() throws IOException, URISyntaxException {
+        return put(in, ugi, delegation, nnRpcAddr, path.getAbsolutePath(), op,
+            permission, overwrite, bufferSize, replication, blockSize);
+      }
+    });
+  }
 
 
-    final String fullpath = path.getAbsolutePath();
+  private Response put(
+      final InputStream in,
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final InetSocketAddress nnRpcAddr,
+      final String fullpath,
+      final PutOpParam op,
+      final PermissionParam permission,
+      final OverwriteParam overwrite,
+      final BufferSizeParam bufferSize,
+      final ReplicationParam replication,
+      final BlockSizeParam blockSize
+      ) throws IOException, URISyntaxException {
     final DataNode datanode = (DataNode)context.getAttribute("datanode");
     final DataNode datanode = (DataNode)context.getAttribute("datanode");
 
 
     switch(op.getValue()) {
     switch(op.getValue()) {
@@ -214,8 +231,6 @@ public class DatanodeWebHdfsMethods {
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }
-      }
-    });
   }
   }
 
 
   /** Handle HTTP POST request for the root for the root. */
   /** Handle HTTP POST request for the root for the root. */
@@ -265,8 +280,21 @@ public class DatanodeWebHdfsMethods {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
       public Response run() throws IOException {
       public Response run() throws IOException {
+        return post(in, ugi, delegation, nnRpcAddr, path.getAbsolutePath(), op,
+            bufferSize);
+      }
+    });
+  }
 
 
-    final String fullpath = path.getAbsolutePath();
+  private Response post(
+      final InputStream in,
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final InetSocketAddress nnRpcAddr,
+      final String fullpath,
+      final PostOpParam op,
+      final BufferSizeParam bufferSize
+      ) throws IOException {
     final DataNode datanode = (DataNode)context.getAttribute("datanode");
     final DataNode datanode = (DataNode)context.getAttribute("datanode");
 
 
     switch(op.getValue()) {
     switch(op.getValue()) {
@@ -292,8 +320,6 @@ public class DatanodeWebHdfsMethods {
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }
-      }
-    });
   }
   }
 
 
   /** Handle HTTP GET request for the root. */
   /** Handle HTTP GET request for the root. */
@@ -348,8 +374,22 @@ public class DatanodeWebHdfsMethods {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
       public Response run() throws IOException {
       public Response run() throws IOException {
+        return get(ugi, delegation, nnRpcAddr, path.getAbsolutePath(), op,
+            offset, length, bufferSize);
+      }
+    });
+  }
 
 
-    final String fullpath = path.getAbsolutePath();
+  private Response get(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final InetSocketAddress nnRpcAddr,
+      final String fullpath,
+      final GetOpParam op,
+      final OffsetParam offset,
+      final LengthParam length,
+      final BufferSizeParam bufferSize
+      ) throws IOException {
     final DataNode datanode = (DataNode)context.getAttribute("datanode");
     final DataNode datanode = (DataNode)context.getAttribute("datanode");
     final Configuration conf = new Configuration(datanode.getConf());
     final Configuration conf = new Configuration(datanode.getConf());
 
 
@@ -412,7 +452,5 @@ public class DatanodeWebHdfsMethods {
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }
-      }
-    });
   }
   }
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalListener.java

@@ -60,5 +60,5 @@ public interface JournalListener {
    * Any IOException thrown from the listener is thrown back in 
    * Any IOException thrown from the listener is thrown back in 
    * {@link JournalProtocol#startLogSegment}
    * {@link JournalProtocol#startLogSegment}
    */
    */
-  public void rollLogs(JournalService service, long txid) throws IOException;
+  public void startLogSegment(JournalService service, long txid) throws IOException;
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/journalservice/JournalService.java

@@ -256,7 +256,7 @@ public class JournalService implements JournalProtocol {
     }
     }
     stateHandler.isStartLogSegmentAllowed();
     stateHandler.isStartLogSegmentAllowed();
     verify(epoch, journalInfo);
     verify(epoch, journalInfo);
-    listener.rollLogs(this, txid);
+    listener.startLogSegment(this, txid);
     stateHandler.startLogSegment();
     stateHandler.startLogSegment();
   }
   }
 
 

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

@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -183,21 +184,9 @@ public class BackupImage extends FSImage {
     }
     }
     
     
     // write to BN's local edit log.
     // write to BN's local edit log.
-    logEditsLocally(firstTxId, numTxns, data);
+    editLog.journal(firstTxId, numTxns, data);
   }
   }
 
 
-  /**
-   * Write the batch of edits to the local copy of the edit logs.
-   */
-  private void logEditsLocally(long firstTxId, int numTxns, byte[] data) {
-    long expectedTxId = editLog.getLastWrittenTxId() + 1;
-    Preconditions.checkState(firstTxId == expectedTxId,
-        "received txid batch starting at %s but expected txn %s",
-        firstTxId, expectedTxId);
-    editLog.setNextTxId(firstTxId + numTxns - 1);
-    editLog.logEdit(data.length, data);
-    editLog.logSync();
-  }
 
 
   /**
   /**
    * Apply the batch of edits to the local namespace.
    * Apply the batch of edits to the local namespace.
@@ -342,28 +331,9 @@ public class BackupImage extends FSImage {
    * This causes the BN to also start the new edit log in its local
    * This causes the BN to also start the new edit log in its local
    * directories.
    * directories.
    */
    */
-  synchronized void namenodeStartedLogSegment(long txid)
-      throws IOException {
-    LOG.info("NameNode started a new log segment at txid " + txid);
-    if (editLog.isSegmentOpen()) {
-      if (editLog.getLastWrittenTxId() == txid - 1) {
-        // We are in sync with the NN, so end and finalize the current segment
-        editLog.endCurrentLogSegment(false);
-      } else {
-        // We appear to have missed some transactions -- the NN probably
-        // lost contact with us temporarily. So, mark the current segment
-        // as aborted.
-        LOG.warn("NN started new log segment at txid " + txid +
-            ", but BN had only written up to txid " +
-            editLog.getLastWrittenTxId() +
-            "in the log segment starting at " + 
-        		editLog.getCurSegmentTxId() + ". Aborting this " +
-        		"log segment.");
-        editLog.abortCurrentLogSegment();
-      }
-    }
-    editLog.setNextTxId(txid);
-    editLog.startLogSegment(txid, false);
+  synchronized void namenodeStartedLogSegment(long txid) throws IOException {
+    editLog.startLogSegment(txid, true);
+
     if (bnState == BNState.DROP_UNTIL_NEXT_ROLL) {
     if (bnState == BNState.DROP_UNTIL_NEXT_ROLL) {
       setState(BNState.JOURNAL_ONLY);
       setState(BNState.JOURNAL_ONLY);
     }
     }

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

@@ -242,7 +242,7 @@ public class BackupNode extends NameNode {
      */
      */
     private void verifyJournalRequest(JournalInfo journalInfo)
     private void verifyJournalRequest(JournalInfo journalInfo)
         throws IOException {
         throws IOException {
-      verifyVersion(journalInfo.getLayoutVersion());
+      verifyLayoutVersion(journalInfo.getLayoutVersion());
       String errorMsg = null;
       String errorMsg = null;
       int expectedNamespaceID = namesystem.getNamespaceInfo().getNamespaceID();
       int expectedNamespaceID = namesystem.getNamespaceInfo().getNamespaceID();
       if (journalInfo.getNamespaceId() != expectedNamespaceID) {
       if (journalInfo.getNamespaceId() != expectedNamespaceID) {
@@ -260,7 +260,7 @@ public class BackupNode extends NameNode {
     }
     }
 
 
     /////////////////////////////////////////////////////
     /////////////////////////////////////////////////////
-    // BackupNodeProtocol implementation for backup node.
+    // JournalProtocol implementation for backup node.
     /////////////////////////////////////////////////////
     /////////////////////////////////////////////////////
     @Override
     @Override
     public void startLogSegment(JournalInfo journalInfo, long epoch,
     public void startLogSegment(JournalInfo journalInfo, long epoch,

+ 86 - 34
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -18,18 +18,19 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
-import java.net.URI;
+
 import java.io.IOException;
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.List;
 import java.util.List;
-import java.lang.reflect.Constructor;
 
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -37,14 +38,34 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.conf.Configuration;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -156,20 +177,6 @@ public class FSEditLog  {
     }
     }
   };
   };
 
 
-  /**
-   * Construct FSEditLog with default configuration, taking editDirs from NNStorage
-   * 
-   * @param storage Storage object used by namenode
-   */
-  @VisibleForTesting
-  FSEditLog(NNStorage storage) throws IOException {
-    Configuration conf = new Configuration();
-    // Make sure the edits dirs are set in the provided configuration object.
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
-        StringUtils.join(storage.getEditsDirectories(), ","));
-    init(conf, storage, FSNamesystem.getNamespaceEditsDirs(conf));
-  }
-
   /**
   /**
    * Constructor for FSEditLog. Underlying journals are constructed, but 
    * Constructor for FSEditLog. Underlying journals are constructed, but 
    * no streams are opened until open() is called.
    * no streams are opened until open() is called.
@@ -269,7 +276,7 @@ public class FSEditLog  {
       IOUtils.closeStream(s);
       IOUtils.closeStream(s);
     }
     }
     
     
-    startLogSegment(segmentTxId, true);
+    startLogSegmentAndWriteHeaderTxn(segmentTxId);
     assert state == State.IN_SEGMENT : "Bad state: " + state;
     assert state == State.IN_SEGMENT : "Bad state: " + state;
   }
   }
   
   
@@ -311,10 +318,12 @@ public class FSEditLog  {
       endCurrentLogSegment(true);
       endCurrentLogSegment(true);
     }
     }
     
     
-    try {
-      journalSet.close();
-    } catch (IOException ioe) {
-      LOG.warn("Error closing journalSet", ioe);
+    if (!journalSet.isEmpty()) {
+      try {
+        journalSet.close();
+      } catch (IOException ioe) {
+        LOG.warn("Error closing journalSet", ioe);
+      }
     }
     }
 
 
     state = State.CLOSED;
     state = State.CLOSED;
@@ -813,9 +822,8 @@ public class FSEditLog  {
   }
   }
   
   
   /**
   /**
-   * Used only by unit tests.
+   * Get all the journals this edit log is currently operating on.
    */
    */
-  @VisibleForTesting
   synchronized List<JournalAndStream> getJournals() {
   synchronized List<JournalAndStream> getJournals() {
     return journalSet.getAllJournalStreams();
     return journalSet.getAllJournalStreams();
   }
   }
@@ -863,18 +871,48 @@ public class FSEditLog  {
     endCurrentLogSegment(true);
     endCurrentLogSegment(true);
     
     
     long nextTxId = getLastWrittenTxId() + 1;
     long nextTxId = getLastWrittenTxId() + 1;
-    startLogSegment(nextTxId, true);
+    startLogSegmentAndWriteHeaderTxn(nextTxId);
     
     
     assert curSegmentTxId == nextTxId;
     assert curSegmentTxId == nextTxId;
     return nextTxId;
     return nextTxId;
   }
   }
+
+  /**
+   * Remote namenode just has started a log segment, start log segment locally.
+   */
+  public synchronized void startLogSegment(long txid, 
+      boolean abortCurrentLogSegment) throws IOException {
+    LOG.info("Namenode started a new log segment at txid " + txid);
+    if (isSegmentOpen()) {
+      if (getLastWrittenTxId() == txid - 1) {
+        //In sync with the NN, so end and finalize the current segment`
+        endCurrentLogSegment(false);
+      } else {
+        //Missed some transactions: probably lost contact with NN temporarily.
+        final String mess = "Cannot start a new log segment at txid " + txid
+            + " since only up to txid " + getLastWrittenTxId()
+            + " have been written in the log segment starting at "
+            + getCurSegmentTxId() + ".";
+        if (abortCurrentLogSegment) {
+          //Mark the current segment as aborted.
+          LOG.warn(mess);
+          abortCurrentLogSegment();
+        } else {
+          throw new IOException(mess);
+        }
+      }
+    }
+    setNextTxId(txid);
+    startLogSegment(txid);
+  }
   
   
   /**
   /**
    * Start writing to the log segment with the given txid.
    * Start writing to the log segment with the given txid.
    * Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state. 
    * Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state. 
    */
    */
-  synchronized void startLogSegment(final long segmentTxId,
-      boolean writeHeaderTxn) throws IOException {
+  private void startLogSegment(final long segmentTxId) throws IOException {
+    assert Thread.holdsLock(this);
+
     LOG.info("Starting log segment at " + segmentTxId);
     LOG.info("Starting log segment at " + segmentTxId);
     Preconditions.checkArgument(segmentTxId > 0,
     Preconditions.checkArgument(segmentTxId > 0,
         "Bad txid: %s", segmentTxId);
         "Bad txid: %s", segmentTxId);
@@ -902,12 +940,15 @@ public class FSEditLog  {
     
     
     curSegmentTxId = segmentTxId;
     curSegmentTxId = segmentTxId;
     state = State.IN_SEGMENT;
     state = State.IN_SEGMENT;
+  }
 
 
-    if (writeHeaderTxn) {
-      logEdit(LogSegmentOp.getInstance(cache.get(),
-          FSEditLogOpCodes.OP_START_LOG_SEGMENT));
-      logSync();
-    }
+  synchronized void startLogSegmentAndWriteHeaderTxn(final long segmentTxId
+      ) throws IOException {
+    startLogSegment(segmentTxId);
+
+    logEdit(LogSegmentOp.getInstance(cache.get(),
+        FSEditLogOpCodes.OP_START_LOG_SEGMENT));
+    logSync();
   }
   }
 
 
   /**
   /**
@@ -1056,6 +1097,17 @@ public class FSEditLog  {
     return null;
     return null;
   }
   }
 
 
+  /** Write the batch of edits to edit log. */
+  public synchronized void journal(long firstTxId, int numTxns, byte[] data) {
+    final long expectedTxId = getLastWrittenTxId() + 1;
+    Preconditions.checkState(firstTxId == expectedTxId,
+        "received txid batch starting at %s but expected txid %s",
+        firstTxId, expectedTxId);
+    setNextTxId(firstTxId + numTxns - 1);
+    logEdit(data.length, data);
+    logSync();
+  }
+
   /**
   /**
    * Write an operation to the edit log. Do not sync to persistent
    * Write an operation to the edit log. Do not sync to persistent
    * store yet.
    * store yet.

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

@@ -823,7 +823,7 @@ public class FSImage implements Closeable {
       storage.writeAll();
       storage.writeAll();
     } finally {
     } finally {
       if (editLogWasOpen) {
       if (editLogWasOpen) {
-        editLog.startLogSegment(imageTxId + 1, true);
+        editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1);
         // Take this opportunity to note the current transaction.
         // Take this opportunity to note the current transaction.
         // Even if the namespace save was cancelled, this marker
         // Even if the namespace save was cancelled, this marker
         // is only used to determine what transaction ID is required
         // is only used to determine what transaction ID is required
@@ -1076,7 +1076,8 @@ public class FSImage implements Closeable {
    */
    */
   static Collection<URI> getCheckpointDirs(Configuration conf,
   static Collection<URI> getCheckpointDirs(Configuration conf,
       String defaultValue) {
       String defaultValue) {
-    Collection<String> dirNames = conf.getStringCollection(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY);
+    Collection<String> dirNames = conf.getTrimmedStringCollection(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY);
     if (dirNames.size() == 0 && defaultValue != null) {
     if (dirNames.size() == 0 && defaultValue != null) {
       dirNames.add(defaultValue);
       dirNames.add(defaultValue);
     }
     }
@@ -1085,8 +1086,8 @@ public class FSImage implements Closeable {
 
 
   static List<URI> getCheckpointEditsDirs(Configuration conf,
   static List<URI> getCheckpointEditsDirs(Configuration conf,
       String defaultName) {
       String defaultName) {
-    Collection<String> dirNames = 
-      conf.getStringCollection(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY);
+    Collection<String> dirNames = conf.getTrimmedStringCollection(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY);
     if (dirNames.size() == 0 && defaultName != null) {
     if (dirNames.size() == 0 && defaultName != null) {
       dirNames.add(defaultName);
       dirNames.add(defaultName);
     }
     }

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

@@ -5051,6 +5051,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       innerinfo.put("adminState", node.getAdminState().toString());
       innerinfo.put("adminState", node.getAdminState().toString());
       innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
       innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
       innerinfo.put("capacity", node.getCapacity());
       innerinfo.put("capacity", node.getCapacity());
+      innerinfo.put("numBlocks", node.numBlocks());
       info.put(node.getHostName(), innerinfo);
       info.put(node.getHostName(), innerinfo);
     }
     }
     return JSON.toString(info);
     return JSON.toString(info);

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

@@ -344,7 +344,7 @@ class FileJournalManager implements JournalManager {
     }
     }
   }
   }
 
 
-  private List<EditLogFile> getLogFiles(long fromTxId) throws IOException {
+  List<EditLogFile> getLogFiles(long fromTxId) throws IOException {
     File currentDir = sd.getCurrentDir();
     File currentDir = sd.getCurrentDir();
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<EditLogFile> logFiles = Lists.newArrayList();
     List<EditLogFile> logFiles = Lists.newArrayList();

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

@@ -18,14 +18,17 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.File;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -49,6 +52,9 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
+import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
 import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
 import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
 import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
@@ -61,6 +67,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -767,9 +775,10 @@ public class NameNode {
       boolean force) {
       boolean force) {
     return initializeSharedEdits(conf, force, false);
     return initializeSharedEdits(conf, force, false);
   }
   }
-  
+
   /**
   /**
-   * Format a new shared edits dir.
+   * Format a new shared edits dir and copy in enough edit log segments so that
+   * the standby NN can start up.
    * 
    * 
    * @param conf configuration
    * @param conf configuration
    * @param force format regardless of whether or not the shared edits dir exists
    * @param force format regardless of whether or not the shared edits dir exists
@@ -803,8 +812,19 @@ public class NameNode {
           existingStorage.getBlockPoolID(),
           existingStorage.getBlockPoolID(),
           existingStorage.getCTime(),
           existingStorage.getCTime(),
           existingStorage.getDistributedUpgradeVersion()));
           existingStorage.getDistributedUpgradeVersion()));
-    } catch (Exception e) {
-      LOG.error("Could not format shared edits dir", e);
+      
+      // Need to make sure the edit log segments are in good shape to initialize
+      // the shared edits dir.
+      fsns.getFSImage().getEditLog().close();
+      fsns.getFSImage().getEditLog().initJournalsForWrite();
+      fsns.getFSImage().getEditLog().recoverUnclosedStreams();
+      
+      if (copyEditLogSegmentsToSharedDir(fsns, sharedEditsDirs,
+          newSharedStorage, conf)) {
+        return true; // aborted
+      }
+    } catch (IOException ioe) {
+      LOG.error("Could not initialize shared edits dir", ioe);
       return true; // aborted
       return true; // aborted
     } finally {
     } finally {
       // Have to unlock storage explicitly for the case when we're running in a
       // Have to unlock storage explicitly for the case when we're running in a
@@ -820,6 +840,44 @@ public class NameNode {
     }
     }
     return false; // did not abort
     return false; // did not abort
   }
   }
+  
+  private static boolean copyEditLogSegmentsToSharedDir(FSNamesystem fsns,
+      Collection<URI> sharedEditsDirs, NNStorage newSharedStorage,
+      Configuration conf) throws FileNotFoundException, IOException {
+    // Copy edit log segments into the new shared edits dir.
+    for (JournalAndStream jas : fsns.getFSImage().getEditLog().getJournals()) {
+      FileJournalManager fjm = null;
+      if (!(jas.getManager() instanceof FileJournalManager)) {
+        LOG.error("Cannot populate shared edits dir from non-file " +
+            "journal manager: " + jas.getManager());
+        return true; // aborted
+      } else {
+        fjm = (FileJournalManager) jas.getManager();
+      }
+      for (EditLogFile elf : fjm.getLogFiles(fsns.getFSImage()
+          .getMostRecentCheckpointTxId())) {
+        File editLogSegment = elf.getFile();
+        for (URI sharedEditsUri : sharedEditsDirs) {
+          StorageDirectory sharedEditsDir = newSharedStorage
+              .getStorageDirectory(sharedEditsUri);
+          File targetFile = new File(sharedEditsDir.getCurrentDir(),
+              editLogSegment.getName());
+          if (!targetFile.exists()) {
+            InputStream in = null;
+            OutputStream out = null;
+            try {
+              in = new FileInputStream(editLogSegment);
+              out = new AtomicFileOutputStream(targetFile);
+              IOUtils.copyBytes(in, out, conf);
+            } finally {
+              IOUtils.cleanup(LOG, in, out);
+            }
+          }
+        }
+      }
+    }
+    return false; // did not abort
+  }
 
 
   private static boolean finalize(Configuration conf,
   private static boolean finalize(Configuration conf,
                                boolean isConfirmationNeeded
                                boolean isConfirmationNeeded

+ 45 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceProtocolService;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceProtocolService;
 import org.apache.hadoop.ha.protocolPB.HAServiceProtocolPB;
 import org.apache.hadoop.ha.protocolPB.HAServiceProtocolPB;
 import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
 import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -121,6 +122,8 @@ import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.util.VersionUtil;
 
 
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.BlockingService;
 
 
@@ -147,6 +150,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
   /** The RPC server that listens to requests from clients */
   /** The RPC server that listens to requests from clients */
   protected final RPC.Server clientRpcServer;
   protected final RPC.Server clientRpcServer;
   protected final InetSocketAddress clientRpcAddress;
   protected final InetSocketAddress clientRpcAddress;
+  
+  private final String minimumDataNodeVersion;
 
 
   public NameNodeRpcServer(Configuration conf, NameNode nn)
   public NameNodeRpcServer(Configuration conf, NameNode nn)
       throws IOException {
       throws IOException {
@@ -261,6 +266,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     // The rpc-server port can be ephemeral... ensure we have the correct info
     // The rpc-server port can be ephemeral... ensure we have the correct info
     this.clientRpcAddress = this.clientRpcServer.getListenerAddress(); 
     this.clientRpcAddress = this.clientRpcServer.getListenerAddress(); 
     nn.setRpcServerAddress(conf, clientRpcAddress);
     nn.setRpcServerAddress(conf, clientRpcAddress);
+    
+    this.minimumDataNodeVersion = conf.get(
+        DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY,
+        DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT);
   }
   }
   
   
   /**
   /**
@@ -326,7 +335,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // NamenodeProtocol
   @Override // NamenodeProtocol
   public NamenodeRegistration register(NamenodeRegistration registration)
   public NamenodeRegistration register(NamenodeRegistration registration)
   throws IOException {
   throws IOException {
-    verifyVersion(registration.getVersion());
+    verifyLayoutVersion(registration.getVersion());
     NamenodeRegistration myRegistration = nn.setRegistration();
     NamenodeRegistration myRegistration = nn.setRegistration();
     namesystem.registerBackupNode(registration, myRegistration);
     namesystem.registerBackupNode(registration, myRegistration);
     return myRegistration;
     return myRegistration;
@@ -829,9 +838,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
 
 
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
-  public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg
-      ) throws IOException {
-    verifyVersion(nodeReg.getVersion());
+  public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
+      throws IOException {
+    verifyLayoutVersion(nodeReg.getVersion());
+    verifySoftwareVersion(nodeReg);
     namesystem.registerDatanode(nodeReg);
     namesystem.registerDatanode(nodeReg);
     return nodeReg;
     return nodeReg;
   }
   }
@@ -916,7 +926,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @throws UnregisteredNodeException if the registration is invalid
    * @throws UnregisteredNodeException if the registration is invalid
    */
    */
   void verifyRequest(NodeRegistration nodeReg) throws IOException {
   void verifyRequest(NodeRegistration nodeReg) throws IOException {
-    verifyVersion(nodeReg.getVersion());
+    verifyLayoutVersion(nodeReg.getVersion());
     if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) {
     if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) {
       LOG.warn("Invalid registrationID - expected: "
       LOG.warn("Invalid registrationID - expected: "
           + namesystem.getRegistrationID() + " received: "
           + namesystem.getRegistrationID() + " received: "
@@ -991,10 +1001,39 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @param version
    * @param version
    * @throws IOException
    * @throws IOException
    */
    */
-  void verifyVersion(int version) throws IOException {
+  void verifyLayoutVersion(int version) throws IOException {
     if (version != HdfsConstants.LAYOUT_VERSION)
     if (version != HdfsConstants.LAYOUT_VERSION)
       throw new IncorrectVersionException(version, "data node");
       throw new IncorrectVersionException(version, "data node");
   }
   }
+  
+  private void verifySoftwareVersion(DatanodeRegistration dnReg)
+      throws IncorrectVersionException {
+    String dnVersion = dnReg.getSoftwareVersion();
+    if (VersionUtil.compareVersions(dnVersion, minimumDataNodeVersion) < 0) {
+      IncorrectVersionException ive = new IncorrectVersionException(
+          minimumDataNodeVersion, dnVersion, "DataNode", "NameNode");
+      LOG.warn(ive.getMessage() + " DN: " + dnReg);
+      throw ive;
+    }
+    String nnVersion = VersionInfo.getVersion();
+    if (!dnVersion.equals(nnVersion)) {
+      String messagePrefix = "Reported DataNode version '" + dnVersion +
+          "' of DN " + dnReg + " does not match NameNode version '" +
+          nnVersion + "'";
+      long nnCTime = nn.getFSImage().getStorage().getCTime();
+      long dnCTime = dnReg.getStorageInfo().getCTime();
+      if (nnCTime != dnCTime) {
+        IncorrectVersionException ive = new IncorrectVersionException(
+            messagePrefix + " and CTime of DN ('" + dnCTime +
+            "') does not match CTime of NN ('" + nnCTime + "')");
+        LOG.warn(ive);
+        throw ive;
+      } else {
+        LOG.info(messagePrefix +
+            ". Note: This is normal during a rolling upgrade.");
+      }
+    }
+  }
 
 
   private static String getClientMachine() {
   private static String getClientMachine() {
     String clientMachine = NamenodeWebHdfsMethods.getRemoteAddress();
     String clientMachine = NamenodeWebHdfsMethods.getRemoteAddress();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.tools.DFSHAAdmin;
 import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
 import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;
@@ -146,8 +147,8 @@ public class BootstrapStandby implements Tool, Configurable {
   
   
   private HAServiceProtocol createHAProtocolProxy()
   private HAServiceProtocol createHAProtocolProxy()
       throws IOException {
       throws IOException {
-    return new NNHAServiceTarget(new HdfsConfiguration(conf),
-        nsId, otherNNId).getProxy(conf, 15000);
+    return new NNHAServiceTarget(new HdfsConfiguration(conf), nsId, otherNNId)
+        .getProxy(conf, 15000);
   }
   }
 
 
   private int doRun() throws IOException {
   private int doRun() throws IOException {
@@ -337,7 +338,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
 
   @Override
   @Override
   public void setConf(Configuration conf) {
   public void setConf(Configuration conf) {
-    this.conf = conf;
+    this.conf = DFSHAAdmin.addSecurityConfiguration(conf);
   }
   }
 
 
   @Override
   @Override

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
@@ -93,14 +94,15 @@ public class ConfiguredFailoverProxyProvider<T> implements
             "for URI " + uri);
             "for URI " + uri);
       }
       }
       
       
-      for (InetSocketAddress address : addressesInNN.values()) {
+      Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
+      for (InetSocketAddress address : addressesOfNns) {
         proxies.add(new AddressRpcProxyPair<T>(address));
         proxies.add(new AddressRpcProxyPair<T>(address));
-        
-        // The client may have a delegation token set for the logical
-        // URI of the cluster. Clone this token to apply to each of the
-        // underlying IPC addresses so that the IPC code can find it.
-        HAUtil.cloneDelegationTokenForLogicalUri(ugi, uri, address);
       }
       }
+
+      // The client may have a delegation token set for the logical
+      // URI of the cluster. Clone this token to apply to each of the
+      // underlying IPC addresses so that the IPC code can find it.
+      HAUtil.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns);
     } catch (IOException e) {
     } catch (IOException e) {
       throw new RuntimeException(e);
       throw new RuntimeException(e);
     }
     }

+ 96 - 37
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -314,8 +314,40 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException, URISyntaxException {
       public Response run() throws IOException, URISyntaxException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         try {
         try {
+          return put(ugi, delegation, username, doAsUser,
+              path.getAbsolutePath(), op, destination, owner, group,
+              permission, overwrite, bufferSize, replication, blockSize,
+              modificationTime, accessTime, renameOptions, createParent,
+              delegationTokenArgument);
+        } finally {
+          REMOTE_ADDRESS.set(null);
+        }
+      }
+    });
+  }
+
+  private Response put(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final PutOpParam op,
+      final DestinationParam destination,
+      final OwnerParam owner,
+      final GroupParam group,
+      final PermissionParam permission,
+      final OverwriteParam overwrite,
+      final BufferSizeParam bufferSize,
+      final ReplicationParam replication,
+      final BlockSizeParam blockSize,
+      final ModificationTimeParam modificationTime,
+      final AccessTimeParam accessTime,
+      final RenameOptionSetParam renameOptions,
+      final CreateParentParam createParent,
+      final TokenArgumentParam delegationTokenArgument
+      ) throws IOException, URISyntaxException {
 
 
-    final String fullpath = path.getAbsolutePath();
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NamenodeProtocols np = namenode.getRpcServer();
     final NamenodeProtocols np = namenode.getRpcServer();
@@ -396,12 +428,6 @@ public class NamenodeWebHdfsMethods {
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }
-
-        } finally {
-          REMOTE_ADDRESS.set(null);
-        }
-      }
-    });
   }
   }
 
 
   /** Handle HTTP POST request for the root. */
   /** Handle HTTP POST request for the root. */
@@ -452,8 +478,24 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException, URISyntaxException {
       public Response run() throws IOException, URISyntaxException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         try {
         try {
+          return post(ugi, delegation, username, doAsUser,
+              path.getAbsolutePath(), op, bufferSize);
+        } finally {
+          REMOTE_ADDRESS.set(null);
+        }
+      }
+    });
+  }
 
 
-    final String fullpath = path.getAbsolutePath();
+  private Response post(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final PostOpParam op,
+      final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
 
 
     switch(op.getValue()) {
     switch(op.getValue()) {
@@ -466,12 +508,6 @@ public class NamenodeWebHdfsMethods {
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }
-
-        } finally {
-          REMOTE_ADDRESS.set(null);
-        }
-      }
-    });
   }
   }
 
 
   /** Handle HTTP GET request for the root. */
   /** Handle HTTP GET request for the root. */
@@ -534,9 +570,28 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException, URISyntaxException {
       public Response run() throws IOException, URISyntaxException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         try {
         try {
+          return get(ugi, delegation, username, doAsUser,
+              path.getAbsolutePath(), op, offset, length, renewer, bufferSize);
+        } finally {
+          REMOTE_ADDRESS.set(null);
+        }
+      }
+    });
+  }
 
 
+  private Response get(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final GetOpParam op,
+      final OffsetParam offset,
+      final LengthParam length,
+      final RenewerParam renewer,
+      final BufferSizeParam bufferSize
+      ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
-    final String fullpath = path.getAbsolutePath();
     final NamenodeProtocols np = namenode.getRpcServer();
     final NamenodeProtocols np = namenode.getRpcServer();
 
 
     switch(op.getValue()) {
     switch(op.getValue()) {
@@ -613,13 +668,7 @@ public class NamenodeWebHdfsMethods {
     }
     }
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
-    }    
-
-        } finally {
-          REMOTE_ADDRESS.set(null);
-        }
-      }
-    });
+    }
   }
   }
 
 
   private static DirectoryListing getDirectoryListing(final NamenodeProtocols np,
   private static DirectoryListing getDirectoryListing(final NamenodeProtocols np,
@@ -712,25 +761,35 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException {
       public Response run() throws IOException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         try {
         try {
-
-        final NameNode namenode = (NameNode)context.getAttribute("name.node");
-        final String fullpath = path.getAbsolutePath();
-
-        switch(op.getValue()) {
-        case DELETE:
-        {
-          final boolean b = namenode.getRpcServer().delete(fullpath, recursive.getValue());
-          final String js = JsonUtil.toJsonString("boolean", b);
-          return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
-        }
-        default:
-          throw new UnsupportedOperationException(op + " is not supported");
-        }
-
+          return delete(ugi, delegation, username, doAsUser,
+              path.getAbsolutePath(), op, recursive);
         } finally {
         } finally {
           REMOTE_ADDRESS.set(null);
           REMOTE_ADDRESS.set(null);
         }
         }
       }
       }
     });
     });
   }
   }
+
+  private Response delete(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final DeleteOpParam op,
+      final RecursiveParam recursive
+      ) throws IOException {
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+
+    switch(op.getValue()) {
+    case DELETE:
+    {
+      final boolean b = namenode.getRpcServer().delete(fullpath, recursive.getValue());
+      final String js = JsonUtil.toJsonString("boolean", b);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
 }
 }

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java

@@ -37,12 +37,14 @@ public class DatanodeRegistration extends DatanodeID
 
 
   private StorageInfo storageInfo;
   private StorageInfo storageInfo;
   private ExportedBlockKeys exportedKeys;
   private ExportedBlockKeys exportedKeys;
+  private String softwareVersion;
 
 
   public DatanodeRegistration(DatanodeID dn, StorageInfo info,
   public DatanodeRegistration(DatanodeID dn, StorageInfo info,
-      ExportedBlockKeys keys) {
+      ExportedBlockKeys keys, String softwareVersion) {
     super(dn);
     super(dn);
     this.storageInfo = info;
     this.storageInfo = info;
     this.exportedKeys = keys;
     this.exportedKeys = keys;
+    this.softwareVersion = softwareVersion;
   }
   }
 
 
   public DatanodeRegistration(String ipAddr, int xferPort) {
   public DatanodeRegistration(String ipAddr, int xferPort) {
@@ -71,6 +73,14 @@ public class DatanodeRegistration extends DatanodeID
   public ExportedBlockKeys getExportedKeys() {
   public ExportedBlockKeys getExportedKeys() {
     return exportedKeys;
     return exportedKeys;
   }
   }
+  
+  public void setSoftwareVersion(String softwareVersion) {
+    this.softwareVersion = softwareVersion;
+  }
+  
+  public String getSoftwareVersion() {
+    return softwareVersion;
+  }
 
 
   @Override // NodeRegistration
   @Override // NodeRegistration
   public int getVersion() {
   public int getVersion() {

+ 10 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.util.VersionInfo;
 
 
 /**
 /**
  * NamespaceInfo is returned by the name-node in reply 
  * NamespaceInfo is returned by the name-node in reply 
@@ -38,6 +39,7 @@ public class NamespaceInfo extends StorageInfo {
   String  buildVersion;
   String  buildVersion;
   int distributedUpgradeVersion;
   int distributedUpgradeVersion;
   String blockPoolID = "";    // id of the block pool
   String blockPoolID = "";    // id of the block pool
+  String softwareVersion;
 
 
   public NamespaceInfo() {
   public NamespaceInfo() {
     super();
     super();
@@ -45,16 +47,18 @@ public class NamespaceInfo extends StorageInfo {
   }
   }
 
 
   public NamespaceInfo(int nsID, String clusterID, String bpID,
   public NamespaceInfo(int nsID, String clusterID, String bpID,
-      long cT, int duVersion, String buildVersion) {
+      long cT, int duVersion, String buildVersion, String softwareVersion) {
     super(HdfsConstants.LAYOUT_VERSION, nsID, clusterID, cT);
     super(HdfsConstants.LAYOUT_VERSION, nsID, clusterID, cT);
     blockPoolID = bpID;
     blockPoolID = bpID;
     this.buildVersion = buildVersion;
     this.buildVersion = buildVersion;
     this.distributedUpgradeVersion = duVersion;
     this.distributedUpgradeVersion = duVersion;
+    this.softwareVersion = softwareVersion;
   }
   }
 
 
   public NamespaceInfo(int nsID, String clusterID, String bpID, 
   public NamespaceInfo(int nsID, String clusterID, String bpID, 
       long cT, int duVersion) {
       long cT, int duVersion) {
-    this(nsID, clusterID, bpID, cT, duVersion, Storage.getBuildVersion());
+    this(nsID, clusterID, bpID, cT, duVersion, Storage.getBuildVersion(),
+        VersionInfo.getVersion());
   }
   }
   
   
   public String getBuildVersion() {
   public String getBuildVersion() {
@@ -68,6 +72,10 @@ public class NamespaceInfo extends StorageInfo {
   public String getBlockPoolID() {
   public String getBlockPoolID() {
     return blockPoolID;
     return blockPoolID;
   }
   }
+  
+  public String getSoftwareVersion() {
+    return softwareVersion;
+  }
 
 
   public String toString(){
   public String toString(){
     return super.toString() + ";bpid=" + blockPoolID;
     return super.toString() + ";bpid=" + blockPoolID;

+ 29 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -29,6 +29,7 @@ import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URL;
+import java.util.Collection;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.StringTokenizer;
 import java.util.StringTokenizer;
@@ -117,8 +118,8 @@ public class WebHdfsFileSystem extends FileSystem
   /** Delegation token kind */
   /** Delegation token kind */
   public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
   public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
   /** Token selector */
   /** Token selector */
-  public static final AbstractDelegationTokenSelector<DelegationTokenIdentifier> DT_SELECTOR
-      = new AbstractDelegationTokenSelector<DelegationTokenIdentifier>(TOKEN_KIND) {};
+  public static final WebHdfsDelegationTokenSelector DT_SELECTOR
+      = new WebHdfsDelegationTokenSelector();
 
 
   private static DelegationTokenRenewer<WebHdfsFileSystem> DT_RENEWER = null;
   private static DelegationTokenRenewer<WebHdfsFileSystem> DT_RENEWER = null;
 
 
@@ -164,7 +165,7 @@ public class WebHdfsFileSystem extends FileSystem
     } catch (URISyntaxException e) {
     } catch (URISyntaxException e) {
       throw new IllegalArgumentException(e);
       throw new IllegalArgumentException(e);
     }
     }
-    this.nnAddr = NetUtils.createSocketAddr(uri.toString());
+    this.nnAddr = NetUtils.createSocketAddrForHost(uri.getHost(), uri.getPort());
     this.workingDir = getHomeDirectory();
     this.workingDir = getHomeDirectory();
 
 
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
@@ -174,12 +175,7 @@ public class WebHdfsFileSystem extends FileSystem
 
 
   protected void initDelegationToken() throws IOException {
   protected void initDelegationToken() throws IOException {
     // look for webhdfs token, then try hdfs
     // look for webhdfs token, then try hdfs
-    final Text serviceName = SecurityUtil.buildTokenService(nnAddr);
-    Token<?> token = DT_SELECTOR.selectToken(serviceName, ugi.getTokens());      
-    if (token == null) {
-      token = DelegationTokenSelector.selectHdfsDelegationToken(
-          nnAddr, ugi, getConf());
-    }
+    Token<?> token = selectDelegationToken();
 
 
     //since we don't already have a token, go get one
     //since we don't already have a token, go get one
     boolean createdToken = false;
     boolean createdToken = false;
@@ -200,6 +196,10 @@ public class WebHdfsFileSystem extends FileSystem
     }
     }
   }
   }
 
 
+  protected Token<DelegationTokenIdentifier> selectDelegationToken() {
+    return DT_SELECTOR.selectToken(getUri(), ugi.getTokens(), getConf());
+  }
+
   @Override
   @Override
   protected int getDefaultPort() {
   protected int getDefaultPort() {
     return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
     return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
@@ -845,4 +845,24 @@ public class WebHdfsFileSystem extends FileSystem
       }
       }
     }
     }
   }
   }
+  
+  private static class WebHdfsDelegationTokenSelector
+  extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+    private static final DelegationTokenSelector hdfsTokenSelector =
+        new DelegationTokenSelector();
+    
+    public WebHdfsDelegationTokenSelector() {
+      super(TOKEN_KIND);
+    }
+    
+    Token<DelegationTokenIdentifier> selectToken(URI nnUri,
+        Collection<Token<?>> tokens, Configuration conf) {
+      Token<DelegationTokenIdentifier> token =
+          selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
+      if (token == null) {
+        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf); 
+      }
+      return token;
+    }
+  }
 }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -33,6 +33,7 @@ message DatanodeRegistrationProto {
   required DatanodeIDProto datanodeID = 1;    // Datanode information
   required DatanodeIDProto datanodeID = 1;    // Datanode information
   required StorageInfoProto storageInfo = 2;  // Node information
   required StorageInfoProto storageInfo = 2;  // Node information
   required ExportedBlockKeysProto keys = 3;   // Block keys
   required ExportedBlockKeysProto keys = 3;   // Block keys
+  required string softwareVersion = 4;        // Software version of the DN, e.g. "2.0.0"
 }
 }
 
 
 /**
 /**

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -303,10 +303,11 @@ message RemoteEditLogManifestProto {
  * Namespace information that describes namespace on a namenode
  * Namespace information that describes namespace on a namenode
  */
  */
 message NamespaceInfoProto {
 message NamespaceInfoProto {
-  required string buildVersion = 1;         // Software build version
+  required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
   required uint32 distUpgradeVersion = 2;   // Distributed upgrade version
   required uint32 distUpgradeVersion = 2;   // Distributed upgrade version
   required string blockPoolID = 3;          // block pool used by the namespace
   required string blockPoolID = 3;          // block pool used by the namespace
-  required StorageInfoProto storageInfo = 4;// Noe information
+  required StorageInfoProto storageInfo = 4;// Node information
+  required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
 }
 }
 
 
 /**
 /**

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

@@ -96,6 +96,7 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -1051,16 +1052,14 @@ public class MiniDFSCluster {
       if(dn == null)
       if(dn == null)
         throw new IOException("Cannot start DataNode in "
         throw new IOException("Cannot start DataNode in "
             + dnConf.get(DFS_DATANODE_DATA_DIR_KEY));
             + dnConf.get(DFS_DATANODE_DATA_DIR_KEY));
-      //NOTE: the following is true if and only if:
-      //      hadoop.security.token.service.use_ip=true
-      //since the HDFS does things based on IP:port, we need to add the mapping
-      //for IP:port to rackId
-      String ipAddr = dn.getXferAddress().getAddress().getHostAddress();
+      //since the HDFS does things based on host|ip:port, we need to add the
+      //mapping for the service to rackId
+      String service =
+          SecurityUtil.buildTokenService(dn.getXferAddress()).toString();
       if (racks != null) {
       if (racks != null) {
-        int port = dn.getXferAddress().getPort();
-        LOG.info("Adding node with IP:port : " + ipAddr + ":" + port +
+        LOG.info("Adding node with service : " + service +
                             " to rack " + racks[i-curDatanodesNum]);
                             " to rack " + racks[i-curDatanodesNum]);
-        StaticMapping.addNodeToRack(ipAddr + ":" + port,
+        StaticMapping.addNodeToRack(service,
                                   racks[i-curDatanodesNum]);
                                   racks[i-curDatanodesNum]);
       }
       }
       dn.runDatanodeDaemon();
       dn.runDatanodeDaemon();

+ 118 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java

@@ -17,24 +17,40 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.*;
+
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.DFSClient;
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.VersionInfo;
+import org.junit.Test;
 
 
 /**
 /**
  * This class tests that a file need not be closed before its
  * This class tests that a file need not be closed before its
  * data can be read by another client.
  * data can be read by another client.
  */
  */
-public class TestDatanodeRegistration extends TestCase {
+public class TestDatanodeRegistration {
+  
+  public static final Log LOG = LogFactory.getLog(TestDatanodeRegistration.class);
 
 
   /**
   /**
    * Regression test for HDFS-894 ensures that, when datanodes
    * Regression test for HDFS-894 ensures that, when datanodes
    * are restarted, the new IPC port is registered with the
    * are restarted, the new IPC port is registered with the
    * namenode.
    * namenode.
    */
    */
+  @Test
   public void testChangeIpcPort() throws Exception {
   public void testChangeIpcPort() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     HdfsConfiguration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
@@ -74,4 +90,102 @@ public class TestDatanodeRegistration extends TestCase {
       }
       }
     }
     }
   }
   }
+  
+  @Test
+  public void testRegistrationWithDifferentSoftwareVersions() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY, "3.0.0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY, "3.0.0");
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
+          .build();
+      
+      NamenodeProtocols rpcServer = cluster.getNameNodeRpc();
+      
+      long nnCTime = cluster.getNamesystem().getFSImage().getStorage().getCTime();
+      StorageInfo mockStorageInfo = mock(StorageInfo.class);
+      doReturn(nnCTime).when(mockStorageInfo).getCTime();
+      
+      DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
+      doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn("fake-storage-id").when(mockDnReg).getStorageID();
+      doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
+      
+      // Should succeed when software versions are the same.
+      doReturn("3.0.0").when(mockDnReg).getSoftwareVersion();
+      rpcServer.registerDatanode(mockDnReg);
+      
+      // Should succeed when software version of DN is above minimum required by NN.
+      doReturn("4.0.0").when(mockDnReg).getSoftwareVersion();
+      rpcServer.registerDatanode(mockDnReg);
+      
+      // Should fail when software version of DN is below minimum required by NN.
+      doReturn("2.0.0").when(mockDnReg).getSoftwareVersion();
+      try {
+        rpcServer.registerDatanode(mockDnReg);
+        fail("Should not have been able to register DN with too-low version.");
+      } catch (IncorrectVersionException ive) {
+        GenericTestUtils.assertExceptionContains(
+            "The reported DataNode version is too low", ive);
+        LOG.info("Got expected exception", ive);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  @Test
+  public void testRegistrationWithDifferentSoftwareVersionsDuringUpgrade()
+      throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY, "1.0.0");
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
+          .build();
+      
+      NamenodeProtocols rpcServer = cluster.getNameNodeRpc();
+      
+      long nnCTime = cluster.getNamesystem().getFSImage().getStorage().getCTime();
+      StorageInfo mockStorageInfo = mock(StorageInfo.class);
+      doReturn(nnCTime).when(mockStorageInfo).getCTime();
+      
+      DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
+      doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn("fake-storage-id").when(mockDnReg).getStorageID();
+      doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
+      
+      // Should succeed when software versions are the same and CTimes are the
+      // same.
+      doReturn(VersionInfo.getVersion()).when(mockDnReg).getSoftwareVersion();
+      rpcServer.registerDatanode(mockDnReg);
+      
+      // Should succeed when software versions are the same and CTimes are
+      // different.
+      doReturn(nnCTime + 1).when(mockStorageInfo).getCTime();
+      rpcServer.registerDatanode(mockDnReg);
+      
+      // Should fail when software version of DN is different from NN and CTimes
+      // are different.
+      doReturn(VersionInfo.getVersion() + ".1").when(mockDnReg).getSoftwareVersion();
+      try {
+        rpcServer.registerDatanode(mockDnReg);
+        fail("Should not have been able to register DN with different software" +
+            " versions and CTimes");
+      } catch (IncorrectVersionException ive) {
+        GenericTestUtils.assertExceptionContains(
+            "does not match CTime of NN", ive);
+        LOG.info("Got expected exception", ive);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }
 }

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpDelegationToken.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -66,4 +67,59 @@ public class TestHftpDelegationToken {
     renewToken.setAccessible(true);
     renewToken.setAccessible(true);
     assertSame("wrong token", token, renewToken.get(fs));
     assertSame("wrong token", token, renewToken.get(fs));
   }
   }
+
+  @Test
+  public void testSelectHdfsDelegationToken() throws Exception {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    Configuration conf = new Configuration();
+    URI hftpUri = URI.create("hftp://localhost:0");
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    Token<?> token = null;
+    
+    // test fallback to hdfs token
+    Token<?> hdfsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
+        new Text("127.0.0.1:8020"));
+    ugi.addToken(hdfsToken);
+    
+    HftpFileSystem fs = (HftpFileSystem) FileSystem.get(hftpUri, conf);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(hdfsToken, token);
+    
+    // test hftp is favored over hdfs
+    Token<?> hftpToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        HftpFileSystem.TOKEN_KIND, new Text("127.0.0.1:0"));
+    ugi.addToken(hftpToken);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(hftpToken, token);
+    
+    // switch to using host-based tokens, no token should match
+    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    token = fs.selectDelegationToken();
+    assertNull(token);
+    
+    // test fallback to hdfs token
+    hdfsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
+        new Text("localhost:8020"));
+    ugi.addToken(hdfsToken);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(hdfsToken, token);
+
+    // test hftp is favored over hdfs
+    hftpToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        HftpFileSystem.TOKEN_KIND, new Text("localhost:0"));
+    ugi.addToken(hftpToken);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(hftpToken, token);
+  }
 }
 }

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java

@@ -25,6 +25,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -234,4 +235,56 @@ public class TestReplaceDatanodeOnFailure {
       Assert.assertEquals(REPLICATION, dfsout.getNumCurrentReplicas());
       Assert.assertEquals(REPLICATION, dfsout.getNumCurrentReplicas());
     }        
     }        
   }
   }
+
+  @Test
+  public void testAppend() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    final short REPLICATION = (short)3;
+    
+    Assert.assertEquals(ReplaceDatanodeOnFailure.DEFAULT, ReplaceDatanodeOnFailure.get(conf));
+
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf
+        ).numDataNodes(1).build();
+
+    try {
+      final DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
+      final Path f = new Path(DIR, "testAppend");
+      
+      {
+        LOG.info("create an empty file " + f);
+        fs.create(f, REPLICATION).close();
+        final FileStatus status = fs.getFileStatus(f);
+        Assert.assertEquals(REPLICATION, status.getReplication());
+        Assert.assertEquals(0L, status.getLen());
+      }
+      
+      
+      final byte[] bytes = new byte[1000];
+      {
+        LOG.info("append " + bytes.length + " bytes to " + f);
+        final FSDataOutputStream out = fs.append(f);
+        out.write(bytes);
+        out.close();
+
+        final FileStatus status = fs.getFileStatus(f);
+        Assert.assertEquals(REPLICATION, status.getReplication());
+        Assert.assertEquals(bytes.length, status.getLen());
+      }
+
+      {
+        LOG.info("append another " + bytes.length + " bytes to " + f);
+        try {
+          final FSDataOutputStream out = fs.append(f);
+          out.write(bytes);
+          out.close();
+
+          Assert.fail();
+        } catch(IOException ioe) {
+          LOG.info("This exception is expected", ioe);
+        }
+      }
+    } finally {
+      if (cluster != null) {cluster.shutdown();}
+    }
+  }
 }
 }

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

@@ -429,12 +429,13 @@ public class TestPBHelper {
     ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
     ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
         getBlockKey(1), keys);
         getBlockKey(1), keys);
     DatanodeRegistration reg = new DatanodeRegistration(dnId,
     DatanodeRegistration reg = new DatanodeRegistration(dnId,
-        new StorageInfo(), expKeys);
+        new StorageInfo(), expKeys, "3.0.0");
     DatanodeRegistrationProto proto = PBHelper.convert(reg);
     DatanodeRegistrationProto proto = PBHelper.convert(reg);
     DatanodeRegistration reg2 = PBHelper.convert(proto);
     DatanodeRegistration reg2 = PBHelper.convert(proto);
     compare(reg.getStorageInfo(), reg2.getStorageInfo());
     compare(reg.getStorageInfo(), reg2.getStorageInfo());
     compare(reg.getExportedKeys(), reg2.getExportedKeys());
     compare(reg.getExportedKeys(), reg2.getExportedKeys());
     compare((DatanodeID)reg, (DatanodeID)reg2);
     compare((DatanodeID)reg, (DatanodeID)reg2);
+    assertEquals(reg.getSoftwareVersion(), reg2.getSoftwareVersion());
   }
   }
   
   
   @Test
   @Test

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SaslInputStream;
 import org.apache.hadoop.security.SaslInputStream;
 import org.apache.hadoop.security.SaslRpcClient;
 import org.apache.hadoop.security.SaslRpcClient;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
@@ -91,10 +92,8 @@ public class TestClientProtocolWithDelegationToken {
     DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner, owner, null);
     DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner, owner, null);
     Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>(
     Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>(
         dtId, sm);
         dtId, sm);
-    Text host = new Text(addr.getAddress().getHostAddress() + ":"
-        + addr.getPort());
-    token.setService(host);
-    LOG.info("Service IP address for token is " + host);
+    SecurityUtil.setTokenService(token, addr);
+    LOG.info("Service for token is " + token.getService());
     current.addToken(token);
     current.addToken(token);
     current.doAs(new PrivilegedExceptionAction<Object>() {
     current.doAs(new PrivilegedExceptionAction<Object>() {
       @Override
       @Override

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -92,6 +92,7 @@ public class TestBlockManager {
       dn.updateHeartbeat(
       dn.updateHeartbeat(
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
+      bm.getDatanodeManager().checkIfClusterIsNowMultiRack(dn);
     }
     }
   }
   }
 
 
@@ -310,6 +311,32 @@ public class TestBlockManager {
         rackB.contains(pipeline[1]));
         rackB.contains(pipeline[1]));
   }
   }
   
   
+  @Test
+  public void testBlocksAreNotUnderreplicatedInSingleRack() throws Exception {
+    List<DatanodeDescriptor> nodes = ImmutableList.of( 
+        new DatanodeDescriptor(new DatanodeID("h1", 5020), "/rackA"),
+        new DatanodeDescriptor(new DatanodeID("h2", 5020), "/rackA"),
+        new DatanodeDescriptor(new DatanodeID("h3", 5020), "/rackA"),
+        new DatanodeDescriptor(new DatanodeID("h4", 5020), "/rackA"),
+        new DatanodeDescriptor(new DatanodeID("h5", 5020), "/rackA"),
+        new DatanodeDescriptor(new DatanodeID("h6", 5020), "/rackA")
+      );
+    addNodes(nodes);
+    List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);;
+    for (int i = 0; i < NUM_TEST_ITERS; i++) {
+      doTestSingleRackClusterIsSufficientlyReplicated(i, origNodes);
+    }
+  }
+  
+  private void doTestSingleRackClusterIsSufficientlyReplicated(int testIndex,
+      List<DatanodeDescriptor> origNodes)
+      throws Exception {
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
+    addBlockOnNodes((long)testIndex, origNodes);
+    bm.processMisReplicatedBlocks();
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
+  }
+  
   
   
   /**
   /**
    * Tell the block manager that replication is completed for the given
    * Tell the block manager that replication is completed for the given

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

@@ -97,7 +97,7 @@ public class TestBlocksWithNotEnoughRacks {
       final FileSystem fs = cluster.getFileSystem();
       final FileSystem fs = cluster.getFileSystem();
       DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
       DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
       ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
       ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
-      DFSTestUtil.waitForReplication(cluster, b, 1, REPLICATION_FACTOR, 1);
+      DFSTestUtil.waitForReplication(cluster, b, 1, REPLICATION_FACTOR, 0);
 
 
       // Add a new datanode on a different rack
       // Add a new datanode on a different rack
       String newRacks[] = {"/rack2"};
       String newRacks[] = {"/rack2"};
@@ -165,7 +165,7 @@ public class TestBlocksWithNotEnoughRacks {
       final FileSystem fs = cluster.getFileSystem();
       final FileSystem fs = cluster.getFileSystem();
       DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
       DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
       ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
       ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
-      DFSTestUtil.waitForReplication(cluster, b, 1, REPLICATION_FACTOR, 1);
+      DFSTestUtil.waitForReplication(cluster, b, 1, REPLICATION_FACTOR, 0);
       
       
       // Add new datanodes on a different rack and increase the
       // Add new datanodes on a different rack and increase the
       // replication factor so the block is underreplicated and make
       // replication factor so the block is underreplicated and make

+ 79 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java

@@ -18,48 +18,105 @@
 
 
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
-import java.net.InetSocketAddress;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.VersionInfo;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
-import org.mockito.Mockito;
-
 
 
 public class TestDatanodeRegister { 
 public class TestDatanodeRegister { 
   public static final Log LOG = LogFactory.getLog(TestDatanodeRegister.class);
   public static final Log LOG = LogFactory.getLog(TestDatanodeRegister.class);
 
 
   // Invalid address
   // Invalid address
-  static final InetSocketAddress INVALID_ADDR =
+  private static final InetSocketAddress INVALID_ADDR =
     new InetSocketAddress("127.0.0.1", 1);
     new InetSocketAddress("127.0.0.1", 1);
-
-  @Test
-  public void testDataNodeRegister() throws Exception {
+  
+  private BPServiceActor actor;
+  NamespaceInfo fakeNsInfo;
+  DNConf mockDnConf;
+  
+  @Before
+  public void setUp() throws IOException {
+    mockDnConf = mock(DNConf.class);
+    doReturn(VersionInfo.getVersion()).when(mockDnConf).getMinimumNameNodeVersion();
+    
     DataNode mockDN = mock(DataNode.class);
     DataNode mockDN = mock(DataNode.class);
-    Mockito.doReturn(true).when(mockDN).shouldRun();
+    doReturn(true).when(mockDN).shouldRun();
+    doReturn(mockDnConf).when(mockDN).getDnConf();
     
     
-    BPOfferService mockBPOS = Mockito.mock(BPOfferService.class);
-    Mockito.doReturn(mockDN).when(mockBPOS).getDataNode();
+    BPOfferService mockBPOS = mock(BPOfferService.class);
+    doReturn(mockDN).when(mockBPOS).getDataNode();
     
     
-    BPServiceActor actor = new BPServiceActor(INVALID_ADDR, mockBPOS);
+    actor = new BPServiceActor(INVALID_ADDR, mockBPOS);
 
 
-    NamespaceInfo fakeNSInfo = mock(NamespaceInfo.class);
-    when(fakeNSInfo.getBuildVersion()).thenReturn("NSBuildVersion");
-    DatanodeProtocolClientSideTranslatorPB fakeDNProt = 
+    fakeNsInfo = mock(NamespaceInfo.class);
+    // Return a a good software version.
+    doReturn(VersionInfo.getVersion()).when(fakeNsInfo).getSoftwareVersion();
+    // Return a good layout version for now.
+    doReturn(HdfsConstants.LAYOUT_VERSION).when(fakeNsInfo).getLayoutVersion();
+    
+    DatanodeProtocolClientSideTranslatorPB fakeDnProt = 
         mock(DatanodeProtocolClientSideTranslatorPB.class);
         mock(DatanodeProtocolClientSideTranslatorPB.class);
-    when(fakeDNProt.versionRequest()).thenReturn(fakeNSInfo);
+    when(fakeDnProt.versionRequest()).thenReturn(fakeNsInfo);
+    actor.setNameNode(fakeDnProt);
+  }
 
 
-    actor.setNameNode( fakeDNProt );
-    try {   
+  @Test
+  public void testSoftwareVersionDifferences() throws Exception {
+    // We expect no exception to be thrown when the software versions match.
+    assertEquals(VersionInfo.getVersion(),
+        actor.retrieveNamespaceInfo().getSoftwareVersion());
+    
+    // We expect no exception to be thrown when the min NN version is below the
+    // reported NN version.
+    doReturn("4.0.0").when(fakeNsInfo).getSoftwareVersion();
+    doReturn("3.0.0").when(mockDnConf).getMinimumNameNodeVersion();
+    assertEquals("4.0.0", actor.retrieveNamespaceInfo().getSoftwareVersion());
+    
+    // When the NN reports a version that's too low, throw an exception.
+    doReturn("3.0.0").when(fakeNsInfo).getSoftwareVersion();
+    doReturn("4.0.0").when(mockDnConf).getMinimumNameNodeVersion();
+    try {
+      actor.retrieveNamespaceInfo();
+      fail("Should have thrown an exception for NN with too-low version");
+    } catch (IncorrectVersionException ive) {
+      GenericTestUtils.assertExceptionContains(
+          "The reported NameNode version is too low", ive);
+      LOG.info("Got expected exception", ive);
+    }
+  }
+  
+  @Test
+  public void testDifferentLayoutVersions() throws Exception {
+    // We expect no exceptions to be thrown when the layout versions match.
+    assertEquals(HdfsConstants.LAYOUT_VERSION,
+        actor.retrieveNamespaceInfo().getLayoutVersion());
+    
+    // We expect an exception to be thrown when the NN reports a layout version
+    // different from that of the DN.
+    doReturn(HdfsConstants.LAYOUT_VERSION * 1000).when(fakeNsInfo)
+        .getLayoutVersion();
+    try {
       actor.retrieveNamespaceInfo();
       actor.retrieveNamespaceInfo();
-      fail("register() did not throw exception! " +
-           "Expected: IncorrectVersionException");
-    } catch (IncorrectVersionException ie) {
-      LOG.info("register() returned correct Exception: IncorrectVersionException");
+      fail("Should have failed to retrieve NS info from DN with bad layout version");
+    } catch (IncorrectVersionException ive) {
+      GenericTestUtils.assertExceptionContains(
+          "Unexpected version of namenode", ive);
+      LOG.info("Got expected exception", ive);
     }
     }
   }
   }
 }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/journalservice/TestJournalService.java

@@ -43,7 +43,7 @@ public class TestJournalService {
   private Configuration conf = new HdfsConfiguration();
   private Configuration conf = new HdfsConfiguration();
   
   
   /**
   /**
-   * Test calls backs {@link JournalListener#rollLogs(JournalService, long)} and
+   * Test calls backs {@link JournalListener#startLogSegment(JournalService, long)} and
    * {@link JournalListener#journal(JournalService, long, int, byte[])} are
    * {@link JournalListener#journal(JournalService, long, int, byte[])} are
    * called.
    * called.
    */
    */
@@ -85,7 +85,7 @@ public class TestJournalService {
    */
    */
   private void verifyRollLogsCallback(JournalService s, JournalListener l)
   private void verifyRollLogsCallback(JournalService s, JournalListener l)
       throws IOException {
       throws IOException {
-    Mockito.verify(l, Mockito.times(1)).rollLogs(Mockito.eq(s), Mockito.anyLong());
+    Mockito.verify(l, Mockito.times(1)).startLogSegment(Mockito.eq(s), Mockito.anyLong());
   }
   }
 
 
   /**
   /**

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

@@ -58,6 +58,7 @@ import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.LogManager;
 
 
@@ -783,6 +784,7 @@ public class NNThroughputBenchmark {
       String hostName = DNS.getDefaultHost("default", "default");
       String hostName = DNS.getDefaultHost("default", "default");
       dnRegistration = new DatanodeRegistration(ipAddr, getNodePort(dnIdx));
       dnRegistration = new DatanodeRegistration(ipAddr, getNodePort(dnIdx));
       dnRegistration.setHostName(hostName);
       dnRegistration.setHostName(hostName);
+      dnRegistration.setSoftwareVersion(VersionInfo.getVersion());
       this.blocks = new ArrayList<Block>(blockCapacity);
       this.blocks = new ArrayList<Block>(blockCapacity);
       this.nrBlocks = 0;
       this.nrBlocks = 0;
     }
     }

+ 22 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -141,6 +141,20 @@ public class TestEditLog extends TestCase {
       }
       }
     }
     }
   }
   }
+  
+  /**
+   * Construct FSEditLog with default configuration, taking editDirs from NNStorage
+   * 
+   * @param storage Storage object used by namenode
+   */
+  private static FSEditLog getFSEditLog(NNStorage storage) throws IOException {
+    Configuration conf = new Configuration();
+    // Make sure the edits dirs are set in the provided configuration object.
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        StringUtils.join(",", storage.getEditsDirectories()));
+    FSEditLog log = new FSEditLog(conf, storage, FSNamesystem.getNamespaceEditsDirs(conf));
+    return log;
+  }
 
 
   /**
   /**
    * Test case for an empty edit log from a prior version of Hadoop.
    * Test case for an empty edit log from a prior version of Hadoop.
@@ -863,7 +877,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
     storage = mockStorageWithEdits(
         "[1,100]|[101,200]|[201,]",
         "[1,100]|[101,200]|[201,]",
         "[1,100]|[101,200]|[201,]");
         "[1,100]|[101,200]|[201,]");
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
         log.getEditLogManifest(1).toString());
@@ -875,7 +889,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
     storage = mockStorageWithEdits(
         "[1,100]|[101,200]",
         "[1,100]|[101,200]",
         "[1,100]|[201,300]|[301,400]"); // nothing starting at 101
         "[1,100]|[201,300]|[301,400]"); // nothing starting at 101
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200], [201,300], [301,400]]",
     assertEquals("[[1,100], [101,200], [201,300], [301,400]]",
         log.getEditLogManifest(1).toString());
         log.getEditLogManifest(1).toString());
@@ -885,7 +899,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
     storage = mockStorageWithEdits(
         "[1,100]|[301,400]", // gap from 101 to 300
         "[1,100]|[301,400]", // gap from 101 to 300
         "[301,400]|[401,500]");
         "[301,400]|[401,500]");
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     log.initJournalsForWrite();
     assertEquals("[[301,400], [401,500]]",
     assertEquals("[[301,400], [401,500]]",
         log.getEditLogManifest(1).toString());
         log.getEditLogManifest(1).toString());
@@ -895,7 +909,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
     storage = mockStorageWithEdits(
         "[1,100]|[101,150]", // short log at 101
         "[1,100]|[101,150]", // short log at 101
         "[1,50]|[101,200]"); // short log at 1
         "[1,50]|[101,200]"); // short log at 1
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
         log.getEditLogManifest(1).toString());
@@ -908,7 +922,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
     storage = mockStorageWithEdits(
         "[1,100]|[101,]", 
         "[1,100]|[101,]", 
         "[1,100]|[101,200]"); 
         "[1,100]|[101,200]"); 
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
         log.getEditLogManifest(1).toString());
@@ -998,7 +1012,7 @@ public class TestEditLog extends TestCase {
                                       Collections.<URI>emptyList(),
                                       Collections.<URI>emptyList(),
                                       editUris);
                                       editUris);
     storage.format(new NamespaceInfo());
     storage.format(new NamespaceInfo());
-    FSEditLog editlog = new FSEditLog(storage);    
+    FSEditLog editlog = getFSEditLog(storage);    
     // open the edit log and add two transactions
     // open the edit log and add two transactions
     // logGenerationStamp is used, simply because it doesn't 
     // logGenerationStamp is used, simply because it doesn't 
     // require complex arguments.
     // require complex arguments.
@@ -1080,7 +1094,7 @@ public class TestEditLog extends TestCase {
                                    new AbortSpec(9, 0),
                                    new AbortSpec(9, 0),
                                    new AbortSpec(10, 1));
                                    new AbortSpec(10, 1));
     long totaltxnread = 0;
     long totaltxnread = 0;
-    FSEditLog editlog = new FSEditLog(storage);
+    FSEditLog editlog = getFSEditLog(storage);
     editlog.initJournalsForWrite();
     editlog.initJournalsForWrite();
     long startTxId = 1;
     long startTxId = 1;
     Iterable<EditLogInputStream> editStreams = editlog.selectInputStreams(startTxId, 
     Iterable<EditLogInputStream> editStreams = editlog.selectInputStreams(startTxId, 
@@ -1130,7 +1144,7 @@ public class TestEditLog extends TestCase {
     assertEquals(1, files.length);
     assertEquals(1, files.length);
     assertTrue(files[0].delete());
     assertTrue(files[0].delete());
     
     
-    FSEditLog editlog = new FSEditLog(storage);
+    FSEditLog editlog = getFSEditLog(storage);
     editlog.initJournalsForWrite();
     editlog.initJournalsForWrite();
     long startTxId = 1;
     long startTxId = 1;
     try {
     try {

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -490,4 +490,46 @@ public class TestNameEditsConfigs {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
+
+  /**
+   * Test dfs.namenode.checkpoint.dir and dfs.namenode.checkpoint.edits.dir
+   * should tolerate white space between values.
+   */
+  @Test
+  public void testCheckPointDirsAreTrimmed() throws Exception {
+    MiniDFSCluster cluster = null;
+    SecondaryNameNode secondary = null;
+    File checkpointNameDir1 = new File(base_dir, "chkptName1");
+    File checkpointEditsDir1 = new File(base_dir, "chkptEdits1");
+    File checkpointNameDir2 = new File(base_dir, "chkptName2");
+    File checkpointEditsDir2 = new File(base_dir, "chkptEdits2");
+    File nameDir = new File(base_dir, "name1");
+    String whiteSpace = "  \n   \n  ";
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getPath());
+    conf.setStrings(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, whiteSpace
+        + checkpointNameDir1.getPath() + whiteSpace, whiteSpace
+        + checkpointNameDir2.getPath() + whiteSpace);
+    conf.setStrings(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
+        whiteSpace + checkpointEditsDir1.getPath() + whiteSpace, whiteSpace
+            + checkpointEditsDir2.getPath() + whiteSpace);
+    cluster = new MiniDFSCluster.Builder(conf).manageNameDfsDirs(false)
+        .numDataNodes(3).build();
+    try {
+      cluster.waitActive();
+      secondary = startSecondaryNameNode(conf);
+      secondary.doCheckpoint();
+      assertTrue(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + " must be trimmed ",
+          checkpointNameDir1.exists());
+      assertTrue(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + " must be trimmed ",
+          checkpointNameDir2.exists());
+      assertTrue(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY
+          + " must be trimmed ", checkpointEditsDir1.exists());
+      assertTrue(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY
+          + " must be trimmed ", checkpointEditsDir2.exists());
+    } finally {
+      secondary.shutdown();
+      cluster.shutdown();
+    }
+  }
 }
 }

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

@@ -101,6 +101,8 @@ public class TestNameNodeMXBean {
         assertTrue(((Long)liveNode.get("nonDfsUsedSpace")) > 0);
         assertTrue(((Long)liveNode.get("nonDfsUsedSpace")) > 0);
         assertTrue(liveNode.containsKey("capacity"));
         assertTrue(liveNode.containsKey("capacity"));
         assertTrue(((Long)liveNode.get("capacity")) > 0);
         assertTrue(((Long)liveNode.get("capacity")) > 0);
+        assertTrue(liveNode.containsKey("numBlocks"));
+        assertTrue(((Long)liveNode.get("numBlocks")) == 0);
       }
       }
       Assert.assertEquals(fsn.getLiveNodes(), alivenodeinfo);
       Assert.assertEquals(fsn.getLiveNodes(), alivenodeinfo);
       // get attribute deadnodeinfo
       // get attribute deadnodeinfo

+ 68 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java

@@ -26,10 +26,12 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.HashSet;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -42,11 +44,14 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -98,6 +103,11 @@ public class TestDelegationTokensWithHA {
   }
   }
 
 
 
 
+  @Before
+  public void prepTest() {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+  }
+  
   @Test
   @Test
   public void testDelegationTokenDFSApi() throws Exception {
   public void testDelegationTokenDFSApi() throws Exception {
     Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
     Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
@@ -185,24 +195,48 @@ public class TestDelegationTokensWithHA {
     URI haUri = new URI("hdfs://my-ha-uri/");
     URI haUri = new URI("hdfs://my-ha-uri/");
     token.setService(HAUtil.buildTokenServiceForLogicalUri(haUri));
     token.setService(HAUtil.buildTokenServiceForLogicalUri(haUri));
     ugi.addToken(token);
     ugi.addToken(token);
-    HAUtil.cloneDelegationTokenForLogicalUri(ugi, haUri, nn0.getNameNodeAddress());
-    HAUtil.cloneDelegationTokenForLogicalUri(ugi, haUri, nn1.getNameNodeAddress());
+
+    Collection<InetSocketAddress> nnAddrs = new HashSet<InetSocketAddress>();
+    nnAddrs.add(nn0.getNameNodeAddress());
+    nnAddrs.add(nn1.getNameNodeAddress());
+    HAUtil.cloneDelegationTokenForLogicalUri(ugi, haUri, nnAddrs);
     
     
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getTokens();
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getTokens();
     assertEquals(3, tokens.size());
     assertEquals(3, tokens.size());
     
     
     LOG.info("Tokens:\n" + Joiner.on("\n").join(tokens));
     LOG.info("Tokens:\n" + Joiner.on("\n").join(tokens));
+    DelegationTokenSelector dts = new DelegationTokenSelector();
     
     
     // check that the token selected for one of the physical IPC addresses
     // check that the token selected for one of the physical IPC addresses
     // matches the one we received
     // matches the one we received
-    InetSocketAddress addr = nn0.getNameNodeAddress();
-    Text ipcDtService = new Text(
-        addr.getAddress().getHostAddress() + ":" + addr.getPort());
-    Token<DelegationTokenIdentifier> token2 =
-        DelegationTokenSelector.selectHdfsDelegationToken(ipcDtService, ugi);
-    assertNotNull(token2);
-    assertArrayEquals(token.getIdentifier(), token2.getIdentifier());
-    assertArrayEquals(token.getPassword(), token2.getPassword());
+    for (InetSocketAddress addr : nnAddrs) {
+      Text ipcDtService = SecurityUtil.buildTokenService(addr);
+      Token<DelegationTokenIdentifier> token2 =
+          dts.selectToken(ipcDtService, ugi.getTokens());
+      assertNotNull(token2);
+      assertArrayEquals(token.getIdentifier(), token2.getIdentifier());
+      assertArrayEquals(token.getPassword(), token2.getPassword());
+    }
+    
+    // switch to host-based tokens, shouldn't match existing tokens 
+    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    for (InetSocketAddress addr : nnAddrs) {
+      Text ipcDtService = SecurityUtil.buildTokenService(addr);
+      Token<DelegationTokenIdentifier> token2 =
+          dts.selectToken(ipcDtService, ugi.getTokens());
+      assertNull(token2);
+    }
+    
+    // reclone the tokens, and see if they match now
+    HAUtil.cloneDelegationTokenForLogicalUri(ugi, haUri, nnAddrs);
+    for (InetSocketAddress addr : nnAddrs) {
+      Text ipcDtService = SecurityUtil.buildTokenService(addr);
+      Token<DelegationTokenIdentifier> token2 =
+          dts.selectToken(ipcDtService, ugi.getTokens());
+      assertNotNull(token2);
+      assertArrayEquals(token.getIdentifier(), token2.getIdentifier());
+      assertArrayEquals(token.getPassword(), token2.getPassword());
+    }    
   }
   }
 
 
   /**
   /**
@@ -212,8 +246,30 @@ public class TestDelegationTokensWithHA {
    */
    */
   @Test
   @Test
   public void testDFSGetCanonicalServiceName() throws Exception {
   public void testDFSGetCanonicalServiceName() throws Exception {
-    assertEquals(fs.getCanonicalServiceName(), 
-        HATestUtil.getLogicalUri(cluster).getHost());
+    URI hAUri = HATestUtil.getLogicalUri(cluster);
+    String haService = HAUtil.buildTokenServiceForLogicalUri(hAUri).toString();
+    assertEquals(haService, dfs.getCanonicalServiceName());
+    Token<?> token = dfs.getDelegationToken(
+        UserGroupInformation.getCurrentUser().getShortUserName());
+    assertEquals(haService, token.getService().toString());
+    // make sure the logical uri is handled correctly
+    token.renew(dfs.getConf());
+    token.cancel(dfs.getConf());
+  }
+  
+  @Test
+  public void testHdfsGetCanonicalServiceName() throws Exception {
+    Configuration conf = dfs.getConf();
+    URI haUri = HATestUtil.getLogicalUri(cluster);
+    AbstractFileSystem afs =  AbstractFileSystem.createFileSystem(haUri, conf);    
+    String haService = HAUtil.buildTokenServiceForLogicalUri(haUri).toString();
+    assertEquals(haService, afs.getCanonicalServiceName());
+    Token<?> token = afs.getDelegationTokens(
+        UserGroupInformation.getCurrentUser().getShortUserName()).get(0);
+    assertEquals(haService, token.getService().toString());
+    // make sure the logical uri is handled correctly
+    token.renew(conf);
+    token.cancel(conf);
   }
   }
   
   
   enum TokenTestAction {
   enum TokenTestAction {

+ 57 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java

@@ -19,17 +19,24 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.URISyntaxException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
+import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
+import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -48,7 +55,10 @@ public class TestInitializeSharedEdits {
   @Before
   @Before
   public void setupCluster() throws IOException {
   public void setupCluster() throws IOException {
     conf = new Configuration();
     conf = new Configuration();
-
+    conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    HAUtil.setAllowStandbyReads(conf, true);
+    
     MiniDFSNNTopology topology = MiniDFSNNTopology.simpleHATopology();
     MiniDFSNNTopology topology = MiniDFSNNTopology.simpleHATopology();
     
     
     cluster = new MiniDFSCluster.Builder(conf)
     cluster = new MiniDFSCluster.Builder(conf)
@@ -56,11 +66,8 @@ public class TestInitializeSharedEdits {
       .numDataNodes(0)
       .numDataNodes(0)
       .build();
       .build();
     cluster.waitActive();
     cluster.waitActive();
-  
-    cluster.shutdownNameNode(0);
-    cluster.shutdownNameNode(1);
-    File sharedEditsDir = new File(cluster.getSharedEditsDir(0, 1));
-    assertTrue(FileUtil.fullyDelete(sharedEditsDir));
+
+    shutdownClusterAndRemoveSharedEditsDir();
   }
   }
   
   
   @After
   @After
@@ -70,8 +77,14 @@ public class TestInitializeSharedEdits {
     }
     }
   }
   }
   
   
-  @Test
-  public void testInitializeSharedEdits() throws Exception {
+  private void shutdownClusterAndRemoveSharedEditsDir() throws IOException {
+    cluster.shutdownNameNode(0);
+    cluster.shutdownNameNode(1);
+    File sharedEditsDir = new File(cluster.getSharedEditsDir(0, 1));
+    assertTrue(FileUtil.fullyDelete(sharedEditsDir));
+  }
+  
+  private void assertCannotStartNameNodes() {
     // Make sure we can't currently start either NN.
     // Make sure we can't currently start either NN.
     try {
     try {
       cluster.restartNameNode(0, false);
       cluster.restartNameNode(0, false);
@@ -89,24 +102,28 @@ public class TestInitializeSharedEdits {
       GenericTestUtils.assertExceptionContains(
       GenericTestUtils.assertExceptionContains(
           "Cannot start an HA namenode with name dirs that need recovery", ioe);
           "Cannot start an HA namenode with name dirs that need recovery", ioe);
     }
     }
-    
-    // Initialize the shared edits dir.
-    assertFalse(NameNode.initializeSharedEdits(conf));
-    
+  }
+  
+  private void assertCanStartHaNameNodes(String pathSuffix)
+      throws ServiceFailedException, IOException, URISyntaxException,
+      InterruptedException {
     // Now should be able to start both NNs. Pass "false" here so that we don't
     // Now should be able to start both NNs. Pass "false" here so that we don't
     // try to waitActive on all NNs, since the second NN doesn't exist yet.
     // try to waitActive on all NNs, since the second NN doesn't exist yet.
     cluster.restartNameNode(0, false);
     cluster.restartNameNode(0, false);
     cluster.restartNameNode(1, true);
     cluster.restartNameNode(1, true);
     
     
     // Make sure HA is working.
     // Make sure HA is working.
-    cluster.transitionToActive(0);
+    cluster.getNameNode(0).getRpcServer().transitionToActive(
+        new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER));
     FileSystem fs = null;
     FileSystem fs = null;
     try {
     try {
+      Path newPath = new Path(TEST_PATH, pathSuffix);
       fs = HATestUtil.configureFailoverFs(cluster, conf);
       fs = HATestUtil.configureFailoverFs(cluster, conf);
-      assertTrue(fs.mkdirs(TEST_PATH));
-      cluster.transitionToStandby(0);
-      cluster.transitionToActive(1);
-      assertTrue(fs.isDirectory(TEST_PATH));
+      assertTrue(fs.mkdirs(newPath));
+      HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
+          cluster.getNameNode(1));
+      assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
+          newPath.toString(), false).isDir());
     } finally {
     } finally {
       if (fs != null) {
       if (fs != null) {
         fs.close();
         fs.close();
@@ -114,6 +131,29 @@ public class TestInitializeSharedEdits {
     }
     }
   }
   }
   
   
+  @Test
+  public void testInitializeSharedEdits() throws Exception {
+    assertCannotStartNameNodes();
+    
+    // Initialize the shared edits dir.
+    assertFalse(NameNode.initializeSharedEdits(cluster.getConfiguration(0)));
+    
+    assertCanStartHaNameNodes("1");
+    
+    // Now that we've done a metadata operation, make sure that deleting and
+    // re-initializing the shared edits dir will let the standby still start.
+    
+    shutdownClusterAndRemoveSharedEditsDir();
+    
+    assertCannotStartNameNodes();
+    
+    // Re-initialize the shared edits dir.
+    assertFalse(NameNode.initializeSharedEdits(cluster.getConfiguration(0)));
+    
+    // Should *still* be able to start both NNs
+    assertCanStartHaNameNodes("2");
+  }
+  
   @Test
   @Test
   public void testDontOverWriteExistingDir() {
   public void testDontOverWriteExistingDir() {
     assertFalse(NameNode.initializeSharedEdits(conf, false));
     assertFalse(NameNode.initializeSharedEdits(conf, false));

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

@@ -34,10 +34,16 @@ import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 
 
 public class TestWebHdfsUrl {
 public class TestWebHdfsUrl {
@@ -90,4 +96,60 @@ public class TestWebHdfsUrl {
   private String generateUrlQueryPrefix(HttpOpParam.Op op, String username) {
   private String generateUrlQueryPrefix(HttpOpParam.Op op, String username) {
     return "op=" + op.toString() + "&user.name=" + username;
     return "op=" + op.toString() + "&user.name=" + username;
   }
   }
+  
+  @Test
+  public void testSelectDelegationToken() throws Exception {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    Configuration conf = new Configuration();
+    URI webHdfsUri = URI.create("webhdfs://localhost:0");
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    Token<?> token = null;
+    
+    // test fallback to hdfs token
+    Token<?> hdfsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
+        new Text("127.0.0.1:8020"));
+    ugi.addToken(hdfsToken);
+    
+    WebHdfsFileSystem fs = (WebHdfsFileSystem) FileSystem.get(webHdfsUri, conf);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(hdfsToken, token);
+    
+    // test webhdfs is favored over hdfs
+    Token<?> webHdfsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        WebHdfsFileSystem.TOKEN_KIND, new Text("127.0.0.1:0"));
+    ugi.addToken(webHdfsToken);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(webHdfsToken, token);
+    
+    // switch to using host-based tokens, no token should match
+    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    token = fs.selectDelegationToken();
+    assertNull(token);
+    
+    // test fallback to hdfs token
+    hdfsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
+        new Text("localhost:8020"));
+    ugi.addToken(hdfsToken);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(hdfsToken, token);
+
+    // test webhdfs is favored over hdfs
+    webHdfsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0],
+        WebHdfsFileSystem.TOKEN_KIND, new Text("localhost:0"));
+    ugi.addToken(webHdfsToken);
+    token = fs.selectDelegationToken();
+    assertNotNull(token);
+    assertEquals(webHdfsToken, token);
+  }
+
 }
 }

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

@@ -52,6 +52,11 @@ Trunk (unreleased changes)
 
 
   BUG FIXES
   BUG FIXES
 
 
+    MAPREDUCE-4149. [Rumen] Rumen fails to parse certain counter
+                    strings. (ravigummadi)
+
+    MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk)
+
     MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
     MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
                     become slow in some cases (ravigummadi).
                     become slow in some cases (ravigummadi).
 
 
@@ -233,6 +238,15 @@ Release 2.0.0 - UNRELEASED
     MAPREDUCE-4108. Fix tests in org.apache.hadoop.util.TestRunJar
     MAPREDUCE-4108. Fix tests in org.apache.hadoop.util.TestRunJar
     (Devaraj K via tgraves)
     (Devaraj K via tgraves)
 
 
+    MAPREDUCE-4107. Fix tests in org.apache.hadoop.ipc.TestSocketFactory
+    (Devaraj K via tgraves)
+
+    MAPREDUCE-4147. YARN should not have a compile-time dependency on HDFS.
+    (tomwhite)
+
+    MAPREDUCE-4008. ResourceManager throws MetricsException on start up 
+    saying QueueMetrics MBean already exists (Devaraj K via tgraves)
+
 Release 0.23.3 - UNRELEASED
 Release 0.23.3 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -247,6 +261,21 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4059. The history server should have a separate pluggable 
     MAPREDUCE-4059. The history server should have a separate pluggable 
     storage/query interface. (Robert Evans via tgraves)
     storage/query interface. (Robert Evans via tgraves)
 
 
+    MAPREDUCE-3942. Randomize master key generation for
+    ApplicationTokenSecretManager and roll it every so often. (Vinod Kumar
+    Vavilapalli via sseth)
+
+    MAPREDUCE-4151. RM scheduler web page should filter apps to those that 
+    are relevant to scheduling (Jason Lowe via tgraves)
+
+    MAPREDUCE-4134. Remove references of mapred.child.ulimit etc. since they
+    are not being used any more (Ravi Prakash via bobby)
+
+    MAPREDUCE-3972. Fix locking and exception issues in JobHistory server.
+    (Robert Joseph Evans via sseth)
+
+    MAPREDUCE-4161. create sockets consistently (Daryn Sharp via bobby)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -310,6 +339,31 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4040. History links should use hostname rather than IP address.
     MAPREDUCE-4040. History links should use hostname rather than IP address.
     (Bhallamudi Venkata Siva Kamesh via sseth)
     (Bhallamudi Venkata Siva Kamesh via sseth)
 
 
+    MAPREDUCE-4099 amendment. ApplicationMaster will remove staging directory
+    after the history service is stopped. (Jason Lowe via sseth)
+
+    MAPREDUCE-3932. Fix the TaskAttempt state machine to handle
+    CONTIANER_LAUNCHED and CONTIANER_LAUNCH_FAILED events in additional
+    states. (Robert Joseph Evans via sseth)
+
+    MAPREDUCE-4140. mapreduce classes incorrectly importing
+    "clover.org.apache.*" classes. (Patrick Hunt via tomwhite)
+
+    MAPREDUCE-4050. For tasks without assigned containers, changes the node
+    text on the UI to N/A instead of a link to null. (Bhallamudi Venkata Siva
+    Kamesh via sseth)
+
+    MAPREDUCE-4128. AM Recovery expects all attempts of a completed task to
+    also be completed. (Bikas Saha via bobby)
+
+    MAPREDUCE-4144. Fix a NPE in the ResourceManager when handling node
+    updates. (Jason Lowe via sseth)
+
+    MAPREDUCE-4156. ant build fails compiling JobInProgress (tgraves)
+
+    MAPREDUCE-4160. some mrv1 ant tests fail with timeout - due to 4156 
+    (tgraves)
+
 Release 0.23.2 - UNRELEASED
 Release 0.23.2 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -285,6 +285,11 @@ public class MRAppMaster extends CompositeService {
     addIfService(containerLauncher);
     addIfService(containerLauncher);
     dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
     dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
 
 
+    // Add the staging directory cleaner before the history server but after
+    // the container allocator so the staging directory is cleaned after
+    // the history has been flushed but before unregistering with the RM.
+    addService(createStagingDirCleaningService());
+
     // Add the JobHistoryEventHandler last so that it is properly stopped first.
     // Add the JobHistoryEventHandler last so that it is properly stopped first.
     // This will guarantee that all history-events are flushed before AM goes
     // This will guarantee that all history-events are flushed before AM goes
     // ahead with shutdown.
     // ahead with shutdown.
@@ -406,13 +411,6 @@ public class MRAppMaster extends CompositeService {
         e.printStackTrace();
         e.printStackTrace();
       }
       }
 
 
-      // Cleanup staging directory
-      try {
-        cleanupStagingDir();
-      } catch(IOException io) {
-        LOG.warn("Failed to delete staging dir", io);
-      }
-
       try {
       try {
         // Stop all services
         // Stop all services
         // This will also send the final report to the ResourceManager
         // This will also send the final report to the ResourceManager
@@ -512,6 +510,10 @@ public class MRAppMaster extends CompositeService {
     return this.jobHistoryEventHandler;
     return this.jobHistoryEventHandler;
   }
   }
 
 
+  protected AbstractService createStagingDirCleaningService() {
+    return new StagingDirCleaningService();
+  }
+
   protected Speculator createSpeculator(Configuration conf, AppContext context) {
   protected Speculator createSpeculator(Configuration conf, AppContext context) {
     Class<? extends Speculator> speculatorClass;
     Class<? extends Speculator> speculatorClass;
 
 
@@ -710,6 +712,22 @@ public class MRAppMaster extends CompositeService {
     }
     }
   }
   }
 
 
+  private final class StagingDirCleaningService extends AbstractService {
+    StagingDirCleaningService() {
+      super(StagingDirCleaningService.class.getName());
+    }
+
+    @Override
+    public synchronized void stop() {
+      try {
+        cleanupStagingDir();
+      } catch (IOException io) {
+        LOG.error("Failed to cleanup staging dir: ", io);
+      }
+      super.stop();
+    }
+  }
+
   private class RunningAppContext implements AppContext {
   private class RunningAppContext implements AppContext {
 
 
     private final Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
     private final Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java

@@ -18,9 +18,11 @@
 
 
 package org.apache.hadoop.mapreduce.v2.app.job;
 package org.apache.hadoop.mapreduce.v2.app.job;
 
 
+import java.io.IOException;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
@@ -71,6 +73,13 @@ public interface Job {
    */
    */
   Path getConfFile();
   Path getConfFile();
   
   
+  /**
+   * @return a parsed version of the config files pointed to by 
+   * {@link #getConfFile()}.
+   * @throws IOException on any error trying to load the conf file. 
+   */
+  Configuration loadConfFile() throws IOException;
+  
   /**
   /**
    * @return the ACLs for this job for each type of JobACL given. 
    * @return the ACLs for this job for each type of JobACL given. 
    */
    */

+ 12 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -37,6 +37,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -347,6 +348,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_DIAGNOSTIC_UPDATE,
                   JobEventType.JOB_DIAGNOSTIC_UPDATE,
                   JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
                   JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
                   JobEventType.INTERNAL_ERROR))
                   JobEventType.INTERNAL_ERROR))
+          .addTransition(JobState.ERROR, JobState.ERROR,
+              JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
           // create the topology tables
           // create the topology tables
           .installTopology();
           .installTopology();
  
  
@@ -1470,4 +1473,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       job.finished(JobState.ERROR);
       job.finished(JobState.ERROR);
     }
     }
   }
   }
+
+  @Override
+  public Configuration loadConfFile() throws IOException {
+    Path confPath = getConfFile();
+    FileContext fc = FileContext.getFileContext(confPath.toUri(), conf);
+    Configuration jobConf = new Configuration(false);
+    jobConf.addResource(fc.open(confPath));
+    return jobConf;
+  }
 }
 }

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

@@ -316,7 +316,9 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_COMMIT_PENDING,
+             // Container launch events can arrive late
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG,
              TaskAttemptEventType.TA_FAILMSG,
              TaskAttemptEventType.TA_TIMED_OUT))
              TaskAttemptEventType.TA_TIMED_OUT))
@@ -338,6 +340,7 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG,
              TaskAttemptEventType.TA_FAILMSG,
              TaskAttemptEventType.TA_TIMED_OUT))
              TaskAttemptEventType.TA_TIMED_OUT))
@@ -359,7 +362,10 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_DONE,
-             TaskAttemptEventType.TA_FAILMSG))
+             TaskAttemptEventType.TA_FAILMSG,
+             // Container launch events can arrive late
+             TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED))
 
 
      // Transitions from KILL_TASK_CLEANUP
      // Transitions from KILL_TASK_CLEANUP
      .addTransition(TaskAttemptState.KILL_TASK_CLEANUP,
      .addTransition(TaskAttemptState.KILL_TASK_CLEANUP,
@@ -377,7 +383,10 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_DONE,
-             TaskAttemptEventType.TA_FAILMSG))
+             TaskAttemptEventType.TA_FAILMSG,
+             // Container launch events can arrive late
+             TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED))
 
 
       // Transitions from SUCCEEDED
       // Transitions from SUCCEEDED
      .addTransition(TaskAttemptState.SUCCEEDED, //only possible for map attempts
      .addTransition(TaskAttemptState.SUCCEEDED, //only possible for map attempts
@@ -405,7 +414,9 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_ASSIGNED,
              TaskAttemptEventType.TA_ASSIGNED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_UPDATE,
+             // Container launch events can arrive late
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG))
              TaskAttemptEventType.TA_FAILMSG))
@@ -420,7 +431,9 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_ASSIGNED,
              TaskAttemptEventType.TA_ASSIGNED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_UPDATE,
+             // Container launch events can arrive late
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG))
              TaskAttemptEventType.TA_FAILMSG))

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -656,6 +656,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
   private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskState taskState) {
   private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskState taskState) {
     TaskFinishedEvent tfe =
     TaskFinishedEvent tfe =
       new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
       new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
+        TypeConverter.fromYarn(task.successfulAttempt),
         task.getFinishTime(task.successfulAttempt),
         task.getFinishTime(task.successfulAttempt),
         TypeConverter.fromYarn(task.taskId.getTaskType()),
         TypeConverter.fromYarn(task.taskId.getTaskType()),
         taskState.toString(),
         taskState.toString(),

+ 6 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.app.rm;
 package org.apache.hadoop.mapreduce.v2.app.rm;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedAction;
 import java.util.Map;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -35,7 +36,6 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -245,11 +245,12 @@ public abstract class RMCommunicator extends AbstractService  {
   }
   }
 
 
   protected AMRMProtocol createSchedulerProxy() {
   protected AMRMProtocol createSchedulerProxy() {
-    final YarnRPC rpc = YarnRPC.create(getConfig());
     final Configuration conf = getConfig();
     final Configuration conf = getConfig();
-    final String serviceAddr = conf.get(
+    final YarnRPC rpc = YarnRPC.create(conf);
+    final InetSocketAddress serviceAddr = conf.getSocketAddr(
         YarnConfiguration.RM_SCHEDULER_ADDRESS,
         YarnConfiguration.RM_SCHEDULER_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS);
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
 
 
     UserGroupInformation currentUser;
     UserGroupInformation currentUser;
     try {
     try {
@@ -279,7 +280,7 @@ public abstract class RMCommunicator extends AbstractService  {
       @Override
       @Override
       public AMRMProtocol run() {
       public AMRMProtocol run() {
         return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class,
         return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class,
-            NetUtils.createSocketAddr(serviceAddr), conf);
+            serviceAddr, conf);
       }
       }
     });
     });
   }
   }

+ 3 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java

@@ -31,7 +31,6 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.Response.Status;
 
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -68,14 +67,11 @@ import com.google.inject.Inject;
 public class AMWebServices {
 public class AMWebServices {
   private final AppContext appCtx;
   private final AppContext appCtx;
   private final App app;
   private final App app;
-  private final Configuration conf;
-
+  
   @Inject
   @Inject
-  public AMWebServices(final App app, final AppContext context,
-      final Configuration conf) {
+  public AMWebServices(final App app, final AppContext context) {
     this.appCtx = context;
     this.appCtx = context;
     this.app = app;
     this.app = app;
-    this.conf = conf;
   }
   }
 
 
   Boolean hasAccess(Job job, HttpServletRequest request) {
   Boolean hasAccess(Job job, HttpServletRequest request) {
@@ -272,7 +268,7 @@ public class AMWebServices {
     checkAccess(job, hsr);
     checkAccess(job, hsr);
     ConfInfo info;
     ConfInfo info;
     try {
     try {
-      info = new ConfInfo(job, this.conf);
+      info = new ConfInfo(job);
     } catch (IOException e) {
     } catch (IOException e) {
       throw new NotFoundException("unable to load configuration for job: "
       throw new NotFoundException("unable to load configuration for job: "
           + jid);
           + jid);

+ 2 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java

@@ -23,7 +23,6 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._TH;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -44,11 +43,9 @@ import com.google.inject.Inject;
  */
  */
 public class ConfBlock extends HtmlBlock {
 public class ConfBlock extends HtmlBlock {
   final AppContext appContext;
   final AppContext appContext;
-  final Configuration conf;
 
 
-  @Inject ConfBlock(AppContext appctx, Configuration conf) {
+  @Inject ConfBlock(AppContext appctx) {
     appContext = appctx;
     appContext = appctx;
-    this.conf = conf;
   }
   }
 
 
   /*
   /*
@@ -71,7 +68,7 @@ public class ConfBlock extends HtmlBlock {
     }
     }
     Path confPath = job.getConfFile();
     Path confPath = job.getConfFile();
     try {
     try {
-      ConfInfo info = new ConfInfo(job, this.conf);
+      ConfInfo info = new ConfInfo(job);
 
 
       html.div().h3(confPath.toString())._();
       html.div().h3(confPath.toString())._();
       TBODY<TABLE<Hamlet>> tbody = html.
       TBODY<TABLE<Hamlet>> tbody = html.

+ 7 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java

@@ -87,9 +87,13 @@ public class TaskPage extends AppView {
           tr().
           tr().
             td(".id", taid).
             td(".id", taid).
             td(".progress", progress).
             td(".progress", progress).
-            td(".state", ta.getState()).
-            td().
-              a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
+            td(".state", ta.getState()).td();
+        if (nodeHttpAddr == null) {
+          nodeTd._("N/A");
+        } else {
+          nodeTd.
+            a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
+        }
         if (containerId != null) {
         if (containerId != null) {
           String containerIdStr = ta.getAssignedContainerIdStr();
           String containerIdStr = ta.getAssignedContainerIdStr();
           nodeTd._(" ").
           nodeTd._(" ").

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

@@ -40,15 +40,11 @@ public class ConfInfo {
   public ConfInfo() {
   public ConfInfo() {
   }
   }
 
 
-  public ConfInfo(Job job, Configuration conf) throws IOException {
+  public ConfInfo(Job job) throws IOException {
 
 
-    Path confPath = job.getConfFile();
     this.property = new ArrayList<ConfEntryInfo>();
     this.property = new ArrayList<ConfEntryInfo>();
-    // Read in the configuration file and put it in a key/value table.
-    FileContext fc = FileContext.getFileContext(confPath.toUri(), conf);
-    Configuration jobConf = new Configuration(false);
-    jobConf.addResource(fc.open(confPath));
-    this.path = confPath.toString();
+    Configuration jobConf = job.loadConfFile();
+    this.path = job.getConfFile().toString();
     for (Map.Entry<String, String> entry : jobConf) {
     for (Map.Entry<String, String> entry : jobConf) {
       this.property.add(new ConfEntryInfo(entry.getKey(), entry.getValue()));
       this.property.add(new ConfEntryInfo(entry.getKey(), entry.getValue()));
     }
     }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -93,7 +93,7 @@ public class TestJobHistoryEventHandler {
 
 
       // First completion event, but min-queue-size for batching flushes is 10
       // First completion event, but min-queue-size for batching flushes is 10
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-          t.taskID, 0, TaskType.MAP, "", null)));
+          t.taskID, null, 0, TaskType.MAP, "", null)));
       verify(mockWriter).flush();
       verify(mockWriter).flush();
 
 
     } finally {
     } finally {
@@ -129,7 +129,7 @@ public class TestJobHistoryEventHandler {
 
 
       for (int i = 0 ; i < 100 ; i++) {
       for (int i = 0 ; i < 100 ; i++) {
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-            t.taskID, 0, TaskType.MAP, "", null)));
+            t.taskID, null, 0, TaskType.MAP, "", null)));
       }
       }
 
 
       handleNextNEvents(jheh, 9);
       handleNextNEvents(jheh, 9);
@@ -174,7 +174,7 @@ public class TestJobHistoryEventHandler {
 
 
       for (int i = 0 ; i < 100 ; i++) {
       for (int i = 0 ; i < 100 ; i++) {
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-            t.taskID, 0, TaskType.MAP, "", null)));
+            t.taskID, null, 0, TaskType.MAP, "", null)));
       }
       }
 
 
       handleNextNEvents(jheh, 9);
       handleNextNEvents(jheh, 9);
@@ -215,7 +215,7 @@ public class TestJobHistoryEventHandler {
 
 
       for (int i = 0 ; i < 100 ; i++) {
       for (int i = 0 ; i < 100 ; i++) {
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-            t.taskID, 0, TaskType.MAP, "", null)));
+            t.taskID, null, 0, TaskType.MAP, "", null)));
       }
       }
       queueEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
       queueEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
           TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters())));
           TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters())));

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

@@ -428,9 +428,13 @@ public class MRApp extends MRAppMaster {
   @Override
   @Override
   protected ContainerAllocator createContainerAllocator(
   protected ContainerAllocator createContainerAllocator(
       ClientService clientService, final AppContext context) {
       ClientService clientService, final AppContext context) {
-    return new ContainerAllocator(){
-      private int containerCount;
-      @Override
+    return new MRAppContainerAllocator();
+  }
+
+  protected class MRAppContainerAllocator implements ContainerAllocator {
+    private int containerCount;
+
+     @Override
       public void handle(ContainerAllocatorEvent event) {
       public void handle(ContainerAllocatorEvent event) {
         ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
         ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
         cId.setApplicationAttemptId(getContext().getApplicationAttemptId());
         cId.setApplicationAttemptId(getContext().getApplicationAttemptId());
@@ -452,7 +456,6 @@ public class MRApp extends MRAppMaster {
             new TaskAttemptContainerAssignedEvent(event.getAttemptID(),
             new TaskAttemptContainerAssignedEvent(event.getAttemptID(),
                 container, null));
                 container, null));
       }
       }
-    };
   }
   }
 
 
   @Override
   @Override

部分文件因文件數量過多而無法顯示