浏览代码

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 "$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:
 $failed_tests"
-    else
-      JIRA_COMMENT="$JIRA_COMMENT
-
-    -1 core tests.  The patch failed the unit tests build"
-    fi
     return 1
   fi
   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
     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
 
     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
     (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-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 
     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
 
   BUG FIXES
@@ -343,6 +350,16 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8264. Remove irritating double double quotes in front of hostname
     (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
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -403,6 +420,9 @@ Release 0.23.3 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-8108. Move method getHostPortString() from NameNode to NetUtils.
+    (Brandon Li via jitendra)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -422,6 +442,17 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8014. ViewFileSystem does not correctly implement getDefaultBlockSize,
     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 
 
   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`
       else
         echo no $command to stop
-        exit 1
       fi
     else
       echo no $command to stop
-      exit 1
     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
 
 # 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
 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.Reader;
 import java.io.Writer;
+import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -68,6 +69,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 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 
    * above means 2, 3, 5, and 7, 8, 9, ...
    */
-  public static class IntegerRanges {
+  public static class IntegerRanges implements Iterable<Integer>{
     private static class Range {
       int start;
       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>();
     
@@ -1025,6 +1073,13 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       return false;
     }
     
+    /**
+     * @return true if there are no values in this range, else false.
+     */
+    public boolean isEmpty() {
+      return ranges == null || ranges.isEmpty();
+    }
+    
     @Override
     public String toString() {
       StringBuilder result = new StringBuilder();
@@ -1041,6 +1096,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       }
       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));
   }
 
+  /**
+   * 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.
    * 

+ 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.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -243,8 +242,8 @@ public class Client {
       this.remoteId = remoteId;
       this.server = remoteId.getAddress();
       if (server.isUnresolved()) {
-        throw NetUtils.wrapException(remoteId.getAddress().getHostName(),
-            remoteId.getAddress().getPort(),
+        throw NetUtils.wrapException(server.getHostName(),
+            server.getPort(),
             null,
             0,
             new UnknownHostException());
@@ -274,9 +273,8 @@ public class Client {
           } catch (IllegalAccessException e) {
             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());
         }
         KerberosInfo krbInfo = SecurityUtil.getKerberosInfo(protocol, conf);
@@ -305,7 +303,7 @@ public class Client {
             + protocol.getSimpleName());
       
       this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
-          remoteId.getAddress().toString() +
+          server.toString() +
           " from " + ((ticket==null)?"an unknown user":ticket.getUserName()));
       this.setDaemon(true);
     }
@@ -751,7 +749,6 @@ public class Client {
       }
     }
 
-    @SuppressWarnings("unused")
     public InetSocketAddress getRemoteAddress() {
       return server;
     }
@@ -1159,7 +1156,7 @@ public class Client {
           call.error.fillInStackTrace();
           throw call.error;
         } else { // local exception
-          InetSocketAddress address = remoteId.getAddress();
+          InetSocketAddress address = connection.getRemoteAddress();
           throw NetUtils.wrapException(address.getHostName(),
                   address.getPort(),
                   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,
       String bindAddress, int port, int numHandlers, int numReaders,
       int queueSizePerHandler, boolean verbose, Configuration conf,
-      SecretManager<? extends TokenIdentifier> secretManager)
+      SecretManager<? extends TokenIdentifier> secretManager,
+      String portRangeConfig)
       throws IOException {
     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 {
@@ -336,15 +338,18 @@ public class ProtobufRpcEngine implements RpcEngine {
      * @param port the port to listen for connections on
      * @param numHandlers the number of method handler threads to run
      * @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,
         Configuration conf, String bindAddress, int port, int numHandlers,
         int numReaders, int queueSizePerHandler, boolean verbose,
-        SecretManager<? extends TokenIdentifier> secretManager)
+        SecretManager<? extends TokenIdentifier> secretManager, 
+        String portRangeConfig)
         throws IOException {
       super(bindAddress, port, null, numHandlers,
           numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
-              .getClass().getName()), secretManager);
+              .getClass().getName()), secretManager, portRangeConfig);
       this.verbose = verbose;  
       registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
           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) 
     throws IOException {
     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. */
@@ -662,7 +663,8 @@ public class RPC {
                                  Object instance, String bindAddress,
                                  int port, Configuration conf) 
     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.
@@ -676,7 +678,7 @@ public class RPC {
     throws IOException {
     
     return getServer(protocol, instance, bindAddress, port, numHandlers, verbose,
-                 conf, null);
+                 conf, null, null);
   }
   
   /** Construct a server for a protocol implementation instance. */
@@ -686,10 +688,20 @@ public class RPC {
                                  boolean verbose, Configuration conf,
                                  SecretManager<? extends TokenIdentifier> secretManager) 
     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)
       .getServer(protocol, instance, bindAddress, port, numHandlers, -1, -1,
-                 verbose, conf, secretManager);
+                 verbose, conf, secretManager, portRangeConfig);
   }
 
   /** Construct a server for a protocol implementation instance. */
@@ -704,7 +716,8 @@ public class RPC {
     
     return getProtocolEngine(protocol, conf)
       .getServer(protocol, instance, bindAddress, port, numHandlers,
-                 numReaders, queueSizePerHandler, verbose, conf, secretManager);
+                 numReaders, queueSizePerHandler, verbose, conf, secretManager,
+                 null);
   }
 
   /** An RPC Server. */
@@ -855,9 +868,10 @@ public class RPC {
                      Class<? extends Writable> paramClass, int handlerCount,
                      int numReaders, int queueSizePerHandler,
                      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,
-            conf, serverName, secretManager);
+            conf, serverName, secretManager, portRangeConfig);
       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)
     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,
                        int port, int numHandlers, int numReaders,
                        int queueSizePerHandler, boolean verbose,
                        Configuration conf, 
-                       SecretManager<? extends TokenIdentifier> secretManager
+                       SecretManager<? extends TokenIdentifier> secretManager,
+                       String portRangeConfig
                        ) 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.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.BytesWritable;
@@ -291,6 +292,7 @@ public abstract class Server {
   protected RpcDetailedMetrics rpcDetailedMetrics;
   
   private Configuration conf;
+  private String portRangeConfig = null;
   private SecretManager<TokenIdentifier> secretManager;
   private ServiceAuthorizationManager serviceAuthorizationManager = new ServiceAuthorizationManager();
 
@@ -323,8 +325,33 @@ public abstract class Server {
    */
   public static void bind(ServerSocket socket, InetSocketAddress address, 
                           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 {
-      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) {
       throw NetUtils.wrapException(null,
           0,
@@ -424,7 +451,7 @@ public abstract class Server {
       acceptChannel.configureBlocking(false);
 
       // 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
       // create a selector;
       selector= Selector.open();
@@ -1725,7 +1752,16 @@ public abstract class Server {
     throws IOException 
   {
     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,
       Class<? extends Writable> rpcRequestClass, int handlerCount,
       int numReaders, int queueSizePerHandler, Configuration conf,
-      String serverName, SecretManager<? extends TokenIdentifier> secretManager)
+      String serverName, SecretManager<? extends TokenIdentifier> secretManager,
+      String portRangeConfig)
     throws IOException {
     this.bindAddress = bindAddress;
     this.conf = conf;
+    this.portRangeConfig = portRangeConfig;
     this.port = port;
     this.rpcRequestClass = rpcRequestClass; 
     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. */
+  @Override
   public RPC.Server getServer(Class<?> protocolClass,
                       Object protocolImpl, String bindAddress, int port,
                       int numHandlers, int numReaders, int queueSizePerHandler,
                       boolean verbose, Configuration conf,
-                      SecretManager<? extends TokenIdentifier> secretManager) 
+                      SecretManager<? extends TokenIdentifier> secretManager,
+                      String portRangeConfig) 
     throws IOException {
     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) 
       throws IOException {
       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 {
        this(null, protocolImpl,  conf,  bindAddress,   port,
                    numHandlers,  numReaders,  queueSizePerHandler,  verbose, 
-                   secretManager);
+                   secretManager, null);
    
     }
     
@@ -381,11 +384,13 @@ public class WritableRpcEngine implements RpcEngine {
     public Server(Class<?> protocolClass, Object protocolImpl,
         Configuration conf, String bindAddress,  int port,
         int numHandlers, int numReaders, int queueSizePerHandler, 
-        boolean verbose, SecretManager<? extends TokenIdentifier> secretManager) 
+        boolean verbose, SecretManager<? extends TokenIdentifier> secretManager,
+        String portRangeConfig) 
         throws IOException {
       super(bindAddress, port, null, numHandlers, numReaders,
           queueSizePerHandler, conf,
-          classNameBase(protocolImpl.getClass().getName()), secretManager);
+          classNameBase(protocolImpl.getClass().getName()), secretManager,
+          portRangeConfig);
 
       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 
    * 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
    * @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) {
     InetSocketAddress addr = server.getListenerAddress();
     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;
   }
@@ -655,7 +660,7 @@ public class NetUtils {
     }
     InetAddress addr = null;
     try {
-      addr = InetAddress.getByName(host);
+      addr = SecurityUtil.getByName(host);
       if (NetworkInterface.getByInetAddress(addr) == null) {
         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 ) {
     int tempIndex = 0;
+    int localRackNode = -1;
     if (reader != null ) {
-      int localRackNode = -1;
       //scan the array to find the local node & local rack node
       for(int i=0; i<nodes.length; i++) {
         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
-    if(tempIndex == 0 && nodes.length != 0) {
+    if(tempIndex == 0 && localRackNode == -1 && nodes.length != 0) {
       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.IOException;
 import java.io.StringWriter;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.regex.Pattern;
 
 import junit.framework.TestCase;
 import static org.junit.Assert.assertArrayEquals;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
 import org.codehaus.jackson.map.ObjectMapper; 
 
 public class TestConfiguration extends TestCase {
@@ -360,6 +366,35 @@ public class TestConfiguration extends TestCase {
     assertEquals(true, range.isIncluded(34));
     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{
     out=new BufferedWriter(new FileWriter(CONFIG));
@@ -604,6 +639,38 @@ public class TestConfiguration extends TestCase {
                  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 {
     out=new BufferedWriter(new FileWriter(CONFIG));
     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.
     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",
                 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.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -237,7 +238,8 @@ public class TestTFileSeqFileComparison extends TestCase {
     public SeqFileAppendable(FileSystem fs, Path path, int osBufferSize,
         String compress, int minBlkSize) throws IOException {
       Configuration conf = new Configuration();
-      conf.setBoolean("hadoop.native.lib", true);
+      conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
+                      true);
 
       CompressionCodec codec = null;
       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.IOException;
 import java.net.ConnectException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.lang.management.ManagementFactory;
 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.RetryProxy;
 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.net.NetUtils;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -257,7 +260,8 @@ public class TestRPC {
     public org.apache.hadoop.ipc.RPC.Server getServer(Class<?> protocol,
         Object instance, String bindAddress, int port, int numHandlers,
         int numReaders, int queueSizePerHandler, boolean verbose, Configuration conf,
-        SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
+        SecretManager<? extends TokenIdentifier> secretManager, 
+        String portRangeConfig) throws IOException {
       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
   public void testAuthorization() throws Exception {
     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 {
     String hadoopUser = getEffectiveUser(user, doAs);
     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);
   }
 
@@ -194,7 +194,7 @@ public class HttpFSServer {
   private FileSystem createFileSystem(Principal user, String doAs) throws IOException, FileSystemAccessException {
     String hadoopUser = getEffectiveUser(user, doAs);
     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);
     FileSystemReleaseFilter.setFileSystem(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;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 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
- * 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/>
  * It provides acces to the server context via the singleton {@link #get}.
  * <p/>
@@ -38,7 +40,8 @@ import java.io.IOException;
  * with <code>httpfs.</code>.
  */
 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.
@@ -67,8 +70,8 @@ public class HttpFSServerWebApp extends ServerWebApp {
   /**
    * 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);
   }
 
@@ -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
   public void init() throws ServerException {
@@ -93,7 +98,8 @@ public class HttpFSServerWebApp extends ServerWebApp {
     SERVER = this;
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     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.
    */

+ 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 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"),
     H02("Kerberos initialization failed, {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}"),
     H06("Property [{0}] not defined in configuration object"),
     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;
 

+ 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;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 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.ServiceException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
@@ -32,6 +34,7 @@ import org.apache.hadoop.util.VersionInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 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";
 
-  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() {
     super(PREFIX);
@@ -102,26 +107,40 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
       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) {
       LOG.debug("  {} = {}", entry.getKey(), entry.getValue());
     }
+    setRequiredServiceHadoopConf(serviceHadoopConf);
 
     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
   public void postInit() throws ServiceException {
     super.postInit();
@@ -166,17 +185,6 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     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 {
     return FileSystem.get(namenodeConf);
   }
@@ -202,16 +210,22 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     Check.notEmpty(user, "user");
     Check.notNull(conf, "conf");
     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 {
-      validateNamenode(new URI(conf.get(NAME_NODE_PROPERTY)).getAuthority());
+      validateNamenode(
+        new URI(conf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)).
+          getAuthority());
       UserGroupInformation ugi = getUGI(user);
       return ugi.doAs(new PrivilegedExceptionAction<T>() {
         public T run() throws Exception {
-          Configuration namenodeConf = createNameNodeConf(conf);
-          FileSystem fs = createFileSystem(namenodeConf);
+          FileSystem fs = createFileSystem(conf);
           Instrumentation instrumentation = getServer().get(Instrumentation.class);
           Instrumentation.Cron cron = instrumentation.createCron();
           try {
@@ -236,13 +250,16 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     throws IOException, FileSystemAccessException {
     Check.notEmpty(user, "user");
     Check.notNull(conf, "conf");
+    if (!conf.getBoolean(FILE_SYSTEM_SERVICE_CREATED, false)) {
+      throw new FileSystemAccessException(FileSystemAccessException.ERROR.H04);
+    }
     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);
       return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
         public FileSystem run() throws Exception {
-          Configuration namenodeConf = createNameNodeConf(conf);
-          return createFileSystem(namenodeConf);
+          return createFileSystem(conf);
         }
       });
     } catch (IOException ex) {
@@ -267,11 +284,11 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     closeFileSystem(fs);
   }
 
-
   @Override
-  public Configuration getDefaultConfiguration() {
-    Configuration conf = new Configuration(false);
+  public Configuration getFileSystemConfiguration() {
+    Configuration conf = new Configuration(true);
     ConfigurationUtils.copy(serviceHadoopConf, conf);
+    conf.setBoolean(FILE_SYSTEM_SERVICE_CREATED, true);
     return conf;
   }
 

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

@@ -153,29 +153,6 @@
     </description>
   </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 -->
 
   <property>
@@ -206,12 +183,4 @@
     </description>
   </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>

+ 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
 
-  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
 
@@ -53,11 +53,11 @@ httpfs.fsAccess.conf:fs.default.name=hdfs://localhost:8021
 +---+
   ...
   <property>
-    <name>fsAccess.proxyuser.#HTTPFSUSER#.hosts</name>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.hosts</name>
     <value>httpfs-host.foo.com</value>
   </property>
   <property>
-    <name>fsAccess.proxyuser.#HTTPFSUSER#.groups</name>
+    <name>hadoop.proxyuser.#HTTPFSUSER#.groups</name>
     <value>*</value>
   </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;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
@@ -70,16 +71,24 @@ public class TestHttpFSFileSystem extends HFSTestCase {
     w.write("secret");
     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);
-    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());
-    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);
     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 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.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.HadoopUsersConfTestHelper;
 import org.apache.hadoop.test.TestDir;
@@ -40,12 +42,15 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
+import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.Writer;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.List;
 
 public class TestHttpFSServer extends HFSTestCase {
 
@@ -54,12 +59,48 @@ public class TestHttpFSServer extends HFSTestCase {
   @TestJetty
   public void server() throws Exception {
     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.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 {
     File homeDir = TestDirHelper.getTestDir();
     Assert.assertTrue(new File(homeDir, "conf").mkdir());
@@ -72,13 +113,29 @@ public class TestHttpFSServer extends HFSTestCase {
     w.write("secret");
     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);
-    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());
-    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);
     os.close();
 
@@ -103,7 +160,8 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
 
     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();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
@@ -112,7 +170,8 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertTrue(line.contains("\"counters\":{"));
 
     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();
     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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.TestHdfsHelper;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.util.Arrays;
 
 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
   @TestDir
   public void simpleSecurity() throws Exception {
@@ -124,7 +144,7 @@ public class TestFileSystemAccessService extends HFSTestCase {
                                                           FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
-    conf.set("server.hadoop.conf:foo", "FOO");
+
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class);
@@ -132,6 +152,32 @@ public class TestFileSystemAccessService extends HFSTestCase {
     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
   @TestDir
   public void inWhitelists() throws Exception {
@@ -188,12 +234,17 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.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);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
-    FileSystem fs = hadoop.createFileSystem("u", TestHdfsHelper.getHdfsConf());
+    FileSystem fs = hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration());
     Assert.assertNotNull(fs);
     fs.mkdirs(new Path("/tmp/foo"));
     hadoop.releaseFileSystem(fs);
@@ -214,6 +265,11 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.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);
     conf.set("server.services", services);
     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];
 
-    hadoop.execute("u", TestHdfsHelper.getHdfsConf(), new FileSystemAccess.FileSystemExecutor<Void>() {
+    hadoop.execute("u", hadoop.getFileSystemConfiguration(), new FileSystemAccess.FileSystemExecutor<Void>() {
       @Override
       public Void execute(FileSystem fs) throws IOException {
         fs.mkdirs(new Path("/tmp/foo"));
@@ -248,14 +304,18 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.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);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     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>() {
       @Override
       public Void execute(FileSystem fs) throws IOException {
@@ -271,6 +331,11 @@ public class TestFileSystemAccessService extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String services = StringUtils.join(",", Arrays.asList(InstrumentationService.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);
     conf.set("server.services", services);
     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];
     try {
-      hadoop.execute("u", TestHdfsHelper.getHdfsConf(), new FileSystemAccess.FileSystemExecutor<Void>() {
+      hadoop.execute("u", hadoop.getFileSystemConfiguration(), new FileSystemAccess.FileSystemExecutor<Void>() {
         @Override
         public Void execute(FileSystem fs) throws IOException {
           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) {
     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 {
       String groups = System.getProperty(HADOOP_USER_PREFIX + user);
       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
     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
 
     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 
     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. 
     (Brandon Li via jitendra)
 
@@ -367,6 +371,25 @@ Release 2.0.0 - UNRELEASED
 
     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
 
     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-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
 
     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
     Lee via szetszwo)
 
+    HDFS-2652. Add support for host-based delegation tokens.  (Daryn Sharp via
+    szetszwo)
+
 Release 0.23.2 - UNRELEASED
 
   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.
 
-"$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.
 # 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.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -391,11 +390,15 @@ public class Hdfs extends AbstractFileSystem {
     return new Path(dfs.getLinkTarget(getUriPath(p)));
   }
   
+  @Override
+  public String getCanonicalServiceName() {
+    return dfs.getCanonicalServiceName();
+  }
+
   @Override //AbstractFileSystem
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
     Token<DelegationTokenIdentifier> result = dfs
         .getDelegationToken(renewer == null ? null : new Text(renewer));
-    result.setService(new Text(this.getCanonicalServiceName()));
     List<Token<?>> tokenList = new ArrayList<Token<?>>();
     tokenList.add(result);
     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;
   }
   
+  /**
+   * 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)
    */

+ 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 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 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 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 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_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 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 long initialFileSize = 0; // at time of file open
   private Progressable progress;
-  private short blockReplication; // replication factor of file
+  private final short blockReplication; // replication factor of file
   
   private class Packet {
     long    seqno;               // sequencenumber of buffer in block
@@ -775,9 +775,13 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private int findNewDatanode(final DatanodeInfo[] original
         ) throws IOException {
       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++) {
         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;
   }        
   
-  /** @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
   public FsStatus getStatus(Path p) throws IOException {
     statistics.incrementReadOps(1);
     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
    * zero.
@@ -848,11 +804,7 @@ public class DistributedFileSystem extends FileSystem {
    */
   @Override
   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 = 
     LogFactory.getLog(HAUtil.class);
   
+  private static final DelegationTokenSelector tokenSelector =
+      new DelegationTokenSelector();
+
   private HAUtil() { /* Hidden constructor */ }
 
   /**
@@ -241,25 +244,28 @@ public class HAUtil {
    * one is found, clone it to also represent the underlying namenode address.
    * @param ugi the UGI to modify
    * @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
    */
   public static void cloneDelegationTokenForLogicalUri(
       UserGroupInformation ugi, URI haUri,
-      InetSocketAddress singleNNAddr) {
-    Text haService = buildTokenServiceForLogicalUri(haUri);
+      Collection<InetSocketAddress> nnAddrs) {
+    Text haService = HAUtil.buildTokenServiceForLogicalUri(haUri);
     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.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.TimeZone;
 
 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.DelegationTokenSelector;
 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.web.URLUtils;
 import org.apache.hadoop.io.Text;
@@ -168,10 +168,7 @@ public class HftpFileSystem extends FileSystem
 
   protected void initDelegationToken() throws IOException {
     // 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
     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
   extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+    private static final DelegationTokenSelector hdfsTokenSelector =
+        new DelegationTokenSelector();
 
     public HftpDelegationTokenSelector() {
       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();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion(),
-        info.getBuildVersion());
+        info.getBuildVersion(), info.getSoftwareVersion());
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -612,13 +612,14 @@ public class PBHelper {
         .newBuilder();
     return builder.setDatanodeID(PBHelper.convert((DatanodeID) registration))
         .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) {
     return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
         PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
-            .getKeys()));
+            .getKeys()), proto.getSoftwareVersion());
   }
 
   public static DatanodeCommand convert(DatanodeCommandProto proto) {
@@ -894,7 +895,8 @@ public class PBHelper {
         .setBlockPoolID(info.getBlockPoolID())
         .setBuildVersion(info.getBuildVersion())
         .setDistUpgradeVersion(info.getDistributedUpgradeVersion())
-        .setStorageInfo(PBHelper.convert((StorageInfo)info)).build();
+        .setStorageInfo(PBHelper.convert((StorageInfo)info))
+        .setSoftwareVersion(info.getSoftwareVersion()).build();
   }
   
   // 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;
 
-import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collection;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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.net.NetUtils;
 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.delegation.AbstractDelegationTokenSelector;
 
@@ -37,32 +37,35 @@ public class DelegationTokenSelector
     extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
   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) {
     // this guesses the remote cluster's rpc service port.
     // 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
     // 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;
     if (nnServiceName != null) {
       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() {

+ 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,
                                              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 = 
       conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 
@@ -2829,7 +2828,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       DatanodeDescriptor cur = it.next();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1) {
+          if (numExpectedReplicas == 1 ||
+              (numExpectedReplicas > 1 &&
+                  !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             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.ReflectionUtils;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.net.InetAddresses;
 
 /**
@@ -126,6 +127,12 @@ public class DatanodeManager {
   /** Ask Datanode only up to this many blocks to delete. */
   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,
       final Namesystem namesystem, final Configuration conf
       ) throws IOException {
@@ -331,6 +338,7 @@ public class DatanodeManager {
 
     host2DatanodeMap.add(node);
     networktopology.add(node);
+    checkIfClusterIsNowMultiRack(node);
 
     if (LOG.isDebugEnabled()) {
       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
    * @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
 public class IncorrectVersionException extends IOException {
   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) {
     this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION);
   }
@@ -40,16 +52,9 @@ public class IncorrectVersionException extends IOException {
   public IncorrectVersionException(int versionReported,
                                    String ofWhat,
                                    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.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 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.protocol.DatanodeCommand;
 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.ipc.RemoteException;
 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.collect.Maps;
@@ -178,17 +179,23 @@ class BPServiceActor implements Runnable {
   private void checkNNVersion(NamespaceInfo nsInfo)
       throws IncorrectVersionException {
     // 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()) {
-      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 +
         " actual "+ nsInfo.getLayoutVersion());
       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_HEARTBEAT_INTERVAL_DEFAULT;
 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.hdfs.DFSConfigKeys;
@@ -58,6 +60,8 @@ class DNConf {
   final long deleteReportInterval;
   final long initialBlockReportDelay;
   final int writePacketSize;
+  
+  final String minimumNameNodeVersion;
 
   public DNConf(Configuration conf) {
     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
@@ -111,5 +115,12 @@ class DNConf {
     this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
         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.setHostName(hostName);
     bpRegistration.setStorageID(getStorageId());
+    bpRegistration.setSoftwareVersion(VersionInfo.getVersion());
 
     StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
     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>() {
       @Override
       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");
 
     switch(op.getValue()) {
@@ -214,8 +231,6 @@ public class DatanodeWebHdfsMethods {
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
-      }
-    });
   }
 
   /** Handle HTTP POST request for the root for the root. */
@@ -265,8 +280,21 @@ public class DatanodeWebHdfsMethods {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       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");
 
     switch(op.getValue()) {
@@ -292,8 +320,6 @@ public class DatanodeWebHdfsMethods {
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
-      }
-    });
   }
 
   /** Handle HTTP GET request for the root. */
@@ -348,8 +374,22 @@ public class DatanodeWebHdfsMethods {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       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 Configuration conf = new Configuration(datanode.getConf());
 
@@ -412,7 +452,5 @@ public class DatanodeWebHdfsMethods {
     default:
       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 
    * {@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();
     verify(epoch, journalInfo);
-    listener.rollLogs(this, txid);
+    listener.startLogSegment(this, txid);
     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.Iterator;
 import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -183,21 +184,9 @@ public class BackupImage extends FSImage {
     }
     
     // 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.
@@ -342,28 +331,9 @@ public class BackupImage extends FSImage {
    * This causes the BN to also start the new edit log in its local
    * 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) {
       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)
         throws IOException {
-      verifyVersion(journalInfo.getLayoutVersion());
+      verifyLayoutVersion(journalInfo.getLayoutVersion());
       String errorMsg = null;
       int expectedNamespaceID = namesystem.getNamespaceInfo().getNamespaceID();
       if (journalInfo.getNamespaceId() != expectedNamespaceID) {
@@ -260,7 +260,7 @@ public class BackupNode extends NameNode {
     }
 
     /////////////////////////////////////////////////////
-    // BackupNodeProtocol implementation for backup node.
+    // JournalProtocol implementation for backup node.
     /////////////////////////////////////////////////////
     @Override
     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;
 
 import static org.apache.hadoop.hdfs.server.common.Util.now;
-import java.net.URI;
+
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 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.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.permission.FsPermission;
 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.server.common.HdfsServerConstants.NamenodeRole;
 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.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.conf.Configuration;
 
 import com.google.common.annotations.VisibleForTesting;
 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 
    * no streams are opened until open() is called.
@@ -269,7 +276,7 @@ public class FSEditLog  {
       IOUtils.closeStream(s);
     }
     
-    startLogSegment(segmentTxId, true);
+    startLogSegmentAndWriteHeaderTxn(segmentTxId);
     assert state == State.IN_SEGMENT : "Bad state: " + state;
   }
   
@@ -311,10 +318,12 @@ public class FSEditLog  {
       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;
@@ -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() {
     return journalSet.getAllJournalStreams();
   }
@@ -863,18 +871,48 @@ public class FSEditLog  {
     endCurrentLogSegment(true);
     
     long nextTxId = getLastWrittenTxId() + 1;
-    startLogSegment(nextTxId, true);
+    startLogSegmentAndWriteHeaderTxn(nextTxId);
     
     assert curSegmentTxId == 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.
    * 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);
     Preconditions.checkArgument(segmentTxId > 0,
         "Bad txid: %s", segmentTxId);
@@ -902,12 +940,15 @@ public class FSEditLog  {
     
     curSegmentTxId = segmentTxId;
     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;
   }
 
+  /** 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
    * 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();
     } finally {
       if (editLogWasOpen) {
-        editLog.startLogSegment(imageTxId + 1, true);
+        editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1);
         // Take this opportunity to note the current transaction.
         // Even if the namespace save was cancelled, this marker
         // 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,
       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) {
       dirNames.add(defaultValue);
     }
@@ -1085,8 +1086,8 @@ public class FSImage implements Closeable {
 
   static List<URI> getCheckpointEditsDirs(Configuration conf,
       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) {
       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("nonDfsUsedSpace", node.getNonDfsUsed());
       innerinfo.put("capacity", node.getCapacity());
+      innerinfo.put("numBlocks", node.numBlocks());
       info.put(node.getHostName(), innerinfo);
     }
     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();
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     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;
 
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.server.common.HdfsServerConstants.NamenodeRole;
 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.BootstrapStandby;
 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.NamenodeRegistration;
 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.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -767,9 +775,10 @@ public class NameNode {
       boolean force) {
     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 force format regardless of whether or not the shared edits dir exists
@@ -803,8 +812,19 @@ public class NameNode {
           existingStorage.getBlockPoolID(),
           existingStorage.getCTime(),
           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
     } finally {
       // 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
   }
+  
+  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,
                                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.protocolPB.HAServiceProtocolPB;
 import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 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.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.util.VersionUtil;
 
 import com.google.protobuf.BlockingService;
 
@@ -147,6 +150,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
   /** The RPC server that listens to requests from clients */
   protected final RPC.Server clientRpcServer;
   protected final InetSocketAddress clientRpcAddress;
+  
+  private final String minimumDataNodeVersion;
 
   public NameNodeRpcServer(Configuration conf, NameNode nn)
       throws IOException {
@@ -261,6 +266,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     // The rpc-server port can be ephemeral... ensure we have the correct info
     this.clientRpcAddress = this.clientRpcServer.getListenerAddress(); 
     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
   public NamenodeRegistration register(NamenodeRegistration registration)
   throws IOException {
-    verifyVersion(registration.getVersion());
+    verifyLayoutVersion(registration.getVersion());
     NamenodeRegistration myRegistration = nn.setRegistration();
     namesystem.registerBackupNode(registration, myRegistration);
     return myRegistration;
@@ -829,9 +838,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
   @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);
     return nodeReg;
   }
@@ -916,7 +926,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @throws UnregisteredNodeException if the registration is invalid
    */
   void verifyRequest(NodeRegistration nodeReg) throws IOException {
-    verifyVersion(nodeReg.getVersion());
+    verifyLayoutVersion(nodeReg.getVersion());
     if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) {
       LOG.warn("Invalid registrationID - expected: "
           + namesystem.getRegistrationID() + " received: "
@@ -991,10 +1001,39 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @param version
    * @throws IOException
    */
-  void verifyVersion(int version) throws IOException {
+  void verifyLayoutVersion(int version) throws IOException {
     if (version != HdfsConstants.LAYOUT_VERSION)
       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() {
     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.protocol.NamenodeProtocol;
 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.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
@@ -146,8 +147,8 @@ public class BootstrapStandby implements Tool, Configurable {
   
   private HAServiceProtocol createHAProtocolProxy()
       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 {
@@ -337,7 +338,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
   @Override
   public void setConf(Configuration conf) {
-    this.conf = conf;
+    this.conf = DFSHAAdmin.addSecurityConfiguration(conf);
   }
 
   @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.URI;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -93,14 +94,15 @@ public class ConfiguredFailoverProxyProvider<T> implements
             "for URI " + uri);
       }
       
-      for (InetSocketAddress address : addressesInNN.values()) {
+      Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
+      for (InetSocketAddress address : addressesOfNns) {
         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) {
       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 {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         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 NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NamenodeProtocols np = namenode.getRpcServer();
@@ -396,12 +428,6 @@ public class NamenodeWebHdfsMethods {
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
-
-        } finally {
-          REMOTE_ADDRESS.set(null);
-        }
-      }
-    });
   }
 
   /** Handle HTTP POST request for the root. */
@@ -452,8 +478,24 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException, URISyntaxException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         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");
 
     switch(op.getValue()) {
@@ -466,12 +508,6 @@ public class NamenodeWebHdfsMethods {
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
-
-        } finally {
-          REMOTE_ADDRESS.set(null);
-        }
-      }
-    });
   }
 
   /** Handle HTTP GET request for the root. */
@@ -534,9 +570,28 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException, URISyntaxException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         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 String fullpath = path.getAbsolutePath();
     final NamenodeProtocols np = namenode.getRpcServer();
 
     switch(op.getValue()) {
@@ -613,13 +668,7 @@ public class NamenodeWebHdfsMethods {
     }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
-    }    
-
-        } finally {
-          REMOTE_ADDRESS.set(null);
-        }
-      }
-    });
+    }
   }
 
   private static DirectoryListing getDirectoryListing(final NamenodeProtocols np,
@@ -712,25 +761,35 @@ public class NamenodeWebHdfsMethods {
       public Response run() throws IOException {
         REMOTE_ADDRESS.set(request.getRemoteAddr());
         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 {
           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 ExportedBlockKeys exportedKeys;
+  private String softwareVersion;
 
   public DatanodeRegistration(DatanodeID dn, StorageInfo info,
-      ExportedBlockKeys keys) {
+      ExportedBlockKeys keys, String softwareVersion) {
     super(dn);
     this.storageInfo = info;
     this.exportedKeys = keys;
+    this.softwareVersion = softwareVersion;
   }
 
   public DatanodeRegistration(String ipAddr, int xferPort) {
@@ -71,6 +73,14 @@ public class DatanodeRegistration extends DatanodeID
   public ExportedBlockKeys getExportedKeys() {
     return exportedKeys;
   }
+  
+  public void setSoftwareVersion(String softwareVersion) {
+    this.softwareVersion = softwareVersion;
+  }
+  
+  public String getSoftwareVersion() {
+    return softwareVersion;
+  }
 
   @Override // NodeRegistration
   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.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.util.VersionInfo;
 
 /**
  * NamespaceInfo is returned by the name-node in reply 
@@ -38,6 +39,7 @@ public class NamespaceInfo extends StorageInfo {
   String  buildVersion;
   int distributedUpgradeVersion;
   String blockPoolID = "";    // id of the block pool
+  String softwareVersion;
 
   public NamespaceInfo() {
     super();
@@ -45,16 +47,18 @@ public class NamespaceInfo extends StorageInfo {
   }
 
   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);
     blockPoolID = bpID;
     this.buildVersion = buildVersion;
     this.distributedUpgradeVersion = duVersion;
+    this.softwareVersion = softwareVersion;
   }
 
   public NamespaceInfo(int nsID, String clusterID, String bpID, 
       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() {
@@ -68,6 +72,10 @@ public class NamespaceInfo extends StorageInfo {
   public String getBlockPoolID() {
     return blockPoolID;
   }
+  
+  public String getSoftwareVersion() {
+    return softwareVersion;
+  }
 
   public String toString(){
     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.URISyntaxException;
 import java.net.URL;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.StringTokenizer;
@@ -117,8 +118,8 @@ public class WebHdfsFileSystem extends FileSystem
   /** Delegation token kind */
   public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
   /** 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;
 
@@ -164,7 +165,7 @@ public class WebHdfsFileSystem extends FileSystem
     } catch (URISyntaxException e) {
       throw new IllegalArgumentException(e);
     }
-    this.nnAddr = NetUtils.createSocketAddr(uri.toString());
+    this.nnAddr = NetUtils.createSocketAddrForHost(uri.getHost(), uri.getPort());
     this.workingDir = getHomeDirectory();
 
     if (UserGroupInformation.isSecurityEnabled()) {
@@ -174,12 +175,7 @@ public class WebHdfsFileSystem extends FileSystem
 
   protected void initDelegationToken() throws IOException {
     // 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
     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
   protected int getDefaultPort() {
     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 StorageInfoProto storageInfo = 2;  // Node information
   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
  */
 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 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.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.util.StringUtils;
@@ -1051,16 +1052,14 @@ public class MiniDFSCluster {
       if(dn == null)
         throw new IOException("Cannot start DataNode in "
             + 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) {
-        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]);
-        StaticMapping.addNodeToRack(ipAddr + ":" + port,
+        StaticMapping.addNodeToRack(service,
                                   racks[i-curDatanodesNum]);
       }
       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;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.*;
+
 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.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
  * 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
    * are restarted, the new IPC port is registered with the
    * namenode.
    */
+  @Test
   public void testChangeIpcPort() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     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.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -66,4 +67,59 @@ public class TestHftpDelegationToken {
     renewToken.setAccessible(true);
     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.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -234,4 +235,56 @@ public class TestReplaceDatanodeOnFailure {
       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,
         getBlockKey(1), keys);
     DatanodeRegistration reg = new DatanodeRegistration(dnId,
-        new StorageInfo(), expKeys);
+        new StorageInfo(), expKeys, "3.0.0");
     DatanodeRegistrationProto proto = PBHelper.convert(reg);
     DatanodeRegistration reg2 = PBHelper.convert(proto);
     compare(reg.getStorageInfo(), reg2.getStorageInfo());
     compare(reg.getExportedKeys(), reg2.getExportedKeys());
     compare((DatanodeID)reg, (DatanodeID)reg2);
+    assertEquals(reg.getSoftwareVersion(), reg2.getSoftwareVersion());
   }
   
   @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.SaslRpcClient;
 import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.Level;
@@ -91,10 +92,8 @@ public class TestClientProtocolWithDelegationToken {
     DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner, owner, null);
     Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>(
         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.doAs(new PrivilegedExceptionAction<Object>() {
       @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(
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
           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]));
   }
   
+  @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

+ 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();
       DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
       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
       String newRacks[] = {"/rack2"};
@@ -165,7 +165,7 @@ public class TestBlocksWithNotEnoughRacks {
       final FileSystem fs = cluster.getFileSystem();
       DFSTestUtil.createFile(fs, filePath, 1L, REPLICATION_FACTOR, 1L);
       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
       // 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;
 
-import java.net.InetSocketAddress;
+import static org.junit.Assert.assertEquals;
 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.LogFactory;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 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.mockito.Mockito;
-
 
 public class TestDatanodeRegister { 
   public static final Log LOG = LogFactory.getLog(TestDatanodeRegister.class);
 
   // Invalid address
-  static final InetSocketAddress INVALID_ADDR =
+  private static final InetSocketAddress INVALID_ADDR =
     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);
-    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);
-    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();
-      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();
   
   /**
-   * 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
    * called.
    */
@@ -85,7 +85,7 @@ public class TestJournalService {
    */
   private void verifyRollLogsCallback(JournalService s, JournalListener l)
       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.security.Groups;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 
@@ -783,6 +784,7 @@ public class NNThroughputBenchmark {
       String hostName = DNS.getDefaultHost("default", "default");
       dnRegistration = new DatanodeRegistration(ipAddr, getNodePort(dnIdx));
       dnRegistration.setHostName(hostName);
+      dnRegistration.setSoftwareVersion(VersionInfo.getVersion());
       this.blocks = new ArrayList<Block>(blockCapacity);
       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.
@@ -863,7 +877,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
         "[1,100]|[101,200]|[201,]",
         "[1,100]|[101,200]|[201,]");
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
@@ -875,7 +889,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
         "[1,100]|[101,200]",
         "[1,100]|[201,300]|[301,400]"); // nothing starting at 101
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200], [201,300], [301,400]]",
         log.getEditLogManifest(1).toString());
@@ -885,7 +899,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
         "[1,100]|[301,400]", // gap from 101 to 300
         "[301,400]|[401,500]");
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     assertEquals("[[301,400], [401,500]]",
         log.getEditLogManifest(1).toString());
@@ -895,7 +909,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
         "[1,100]|[101,150]", // short log at 101
         "[1,50]|[101,200]"); // short log at 1
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
@@ -908,7 +922,7 @@ public class TestEditLog extends TestCase {
     storage = mockStorageWithEdits(
         "[1,100]|[101,]", 
         "[1,100]|[101,200]"); 
-    log = new FSEditLog(storage);
+    log = getFSEditLog(storage);
     log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
@@ -998,7 +1012,7 @@ public class TestEditLog extends TestCase {
                                       Collections.<URI>emptyList(),
                                       editUris);
     storage.format(new NamespaceInfo());
-    FSEditLog editlog = new FSEditLog(storage);    
+    FSEditLog editlog = getFSEditLog(storage);    
     // open the edit log and add two transactions
     // logGenerationStamp is used, simply because it doesn't 
     // require complex arguments.
@@ -1080,7 +1094,7 @@ public class TestEditLog extends TestCase {
                                    new AbortSpec(9, 0),
                                    new AbortSpec(10, 1));
     long totaltxnread = 0;
-    FSEditLog editlog = new FSEditLog(storage);
+    FSEditLog editlog = getFSEditLog(storage);
     editlog.initJournalsForWrite();
     long startTxId = 1;
     Iterable<EditLogInputStream> editStreams = editlog.selectInputStreams(startTxId, 
@@ -1130,7 +1144,7 @@ public class TestEditLog extends TestCase {
     assertEquals(1, files.length);
     assertTrue(files[0].delete());
     
-    FSEditLog editlog = new FSEditLog(storage);
+    FSEditLog editlog = getFSEditLog(storage);
     editlog.initJournalsForWrite();
     long startTxId = 1;
     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();
     }
   }
+
+  /**
+   * 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(liveNode.containsKey("capacity"));
         assertTrue(((Long)liveNode.get("capacity")) > 0);
+        assertTrue(liveNode.containsKey("numBlocks"));
+        assertTrue(((Long)liveNode.get("numBlocks")) == 0);
       }
       Assert.assertEquals(fsn.getLiveNodes(), alivenodeinfo);
       // 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.security.PrivilegedExceptionAction;
 import java.util.Collection;
+import java.util.HashSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 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.NameNodeAdapter;
 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.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -98,6 +103,11 @@ public class TestDelegationTokensWithHA {
   }
 
 
+  @Before
+  public void prepTest() {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+  }
+  
   @Test
   public void testDelegationTokenDFSApi() throws Exception {
     Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
@@ -185,24 +195,48 @@ public class TestDelegationTokensWithHA {
     URI haUri = new URI("hdfs://my-ha-uri/");
     token.setService(HAUtil.buildTokenServiceForLogicalUri(haUri));
     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();
     assertEquals(3, tokens.size());
     
     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
     // 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
   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 {

+ 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.IOException;
+import java.net.URISyntaxException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 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.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -48,7 +55,10 @@ public class TestInitializeSharedEdits {
   @Before
   public void setupCluster() throws IOException {
     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();
     
     cluster = new MiniDFSCluster.Builder(conf)
@@ -56,11 +66,8 @@ public class TestInitializeSharedEdits {
       .numDataNodes(0)
       .build();
     cluster.waitActive();
-  
-    cluster.shutdownNameNode(0);
-    cluster.shutdownNameNode(1);
-    File sharedEditsDir = new File(cluster.getSharedEditsDir(0, 1));
-    assertTrue(FileUtil.fullyDelete(sharedEditsDir));
+
+    shutdownClusterAndRemoveSharedEditsDir();
   }
   
   @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.
     try {
       cluster.restartNameNode(0, false);
@@ -89,24 +102,28 @@ public class TestInitializeSharedEdits {
       GenericTestUtils.assertExceptionContains(
           "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
     // try to waitActive on all NNs, since the second NN doesn't exist yet.
     cluster.restartNameNode(0, false);
     cluster.restartNameNode(1, true);
     
     // Make sure HA is working.
-    cluster.transitionToActive(0);
+    cluster.getNameNode(0).getRpcServer().transitionToActive(
+        new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER));
     FileSystem fs = null;
     try {
+      Path newPath = new Path(TEST_PATH, pathSuffix);
       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 {
       if (fs != null) {
         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
   public void testDontOverWriteExistingDir() {
     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.PutOpParam;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.junit.Assert;
 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;
 
 public class TestWebHdfsUrl {
@@ -90,4 +96,60 @@ public class TestWebHdfsUrl {
   private String generateUrlQueryPrefix(HttpOpParam.Op op, String 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
 
+    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
                     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
     (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
 
   INCOMPATIBLE CHANGES
@@ -247,6 +261,21 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4059. The history server should have a separate pluggable 
     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
 
   BUG FIXES
@@ -310,6 +339,31 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4040. History links should use hostname rather than IP address.
     (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
 
   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);
     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.
     // This will guarantee that all history-events are flushed before AM goes
     // ahead with shutdown.
@@ -406,13 +411,6 @@ public class MRAppMaster extends CompositeService {
         e.printStackTrace();
       }
 
-      // Cleanup staging directory
-      try {
-        cleanupStagingDir();
-      } catch(IOException io) {
-        LOG.warn("Failed to delete staging dir", io);
-      }
-
       try {
         // Stop all services
         // This will also send the final report to the ResourceManager
@@ -512,6 +510,10 @@ public class MRAppMaster extends CompositeService {
     return this.jobHistoryEventHandler;
   }
 
+  protected AbstractService createStagingDirCleaningService() {
+    return new StagingDirCleaningService();
+  }
+
   protected Speculator createSpeculator(Configuration conf, AppContext context) {
     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 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;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobACL;
@@ -71,6 +73,13 @@ public interface Job {
    */
   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. 
    */

+ 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.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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_TASK_ATTEMPT_FETCH_FAILURE,
                   JobEventType.INTERNAL_ERROR))
+          .addTransition(JobState.ERROR, JobState.ERROR,
+              JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
           // create the topology tables
           .installTopology();
  
@@ -1470,4 +1473,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       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_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
+             // Container launch events can arrive late
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG,
              TaskAttemptEventType.TA_TIMED_OUT))
@@ -338,6 +340,7 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG,
              TaskAttemptEventType.TA_TIMED_OUT))
@@ -359,7 +362,10 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              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
      .addTransition(TaskAttemptState.KILL_TASK_CLEANUP,
@@ -377,7 +383,10 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_UPDATE,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              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
      .addTransition(TaskAttemptState.SUCCEEDED, //only possible for map attempts
@@ -405,7 +414,9 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_ASSIGNED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_UPDATE,
+             // Container launch events can arrive late
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_DONE,
              TaskAttemptEventType.TA_FAILMSG))
@@ -420,7 +431,9 @@ public abstract class TaskAttemptImpl implements
              TaskAttemptEventType.TA_ASSIGNED,
              TaskAttemptEventType.TA_CONTAINER_COMPLETED,
              TaskAttemptEventType.TA_UPDATE,
+             // Container launch events can arrive late
              TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
              TaskAttemptEventType.TA_COMMIT_PENDING,
              TaskAttemptEventType.TA_DONE,
              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) {
     TaskFinishedEvent tfe =
       new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
+        TypeConverter.fromYarn(task.successfulAttempt),
         task.getFinishTime(task.successfulAttempt),
         TypeConverter.fromYarn(task.taskId.getTaskType()),
         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;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.util.Map;
 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.job.Job;
 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.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -245,11 +245,12 @@ public abstract class RMCommunicator extends AbstractService  {
   }
 
   protected AMRMProtocol createSchedulerProxy() {
-    final YarnRPC rpc = YarnRPC.create(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.DEFAULT_RM_SCHEDULER_ADDRESS);
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
 
     UserGroupInformation currentUser;
     try {
@@ -279,7 +280,7 @@ public abstract class RMCommunicator extends AbstractService  {
       @Override
       public AMRMProtocol run() {
         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.Response.Status;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -68,14 +67,11 @@ import com.google.inject.Inject;
 public class AMWebServices {
   private final AppContext appCtx;
   private final App app;
-  private final Configuration conf;
-
+  
   @Inject
-  public AMWebServices(final App app, final AppContext context,
-      final Configuration conf) {
+  public AMWebServices(final App app, final AppContext context) {
     this.appCtx = context;
     this.app = app;
-    this.conf = conf;
   }
 
   Boolean hasAccess(Job job, HttpServletRequest request) {
@@ -272,7 +268,7 @@ public class AMWebServices {
     checkAccess(job, hsr);
     ConfInfo info;
     try {
-      info = new ConfInfo(job, this.conf);
+      info = new ConfInfo(job);
     } catch (IOException e) {
       throw new NotFoundException("unable to load configuration for job: "
           + 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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -44,11 +43,9 @@ import com.google.inject.Inject;
  */
 public class ConfBlock extends HtmlBlock {
   final AppContext appContext;
-  final Configuration conf;
 
-  @Inject ConfBlock(AppContext appctx, Configuration conf) {
+  @Inject ConfBlock(AppContext appctx) {
     appContext = appctx;
-    this.conf = conf;
   }
 
   /*
@@ -71,7 +68,7 @@ public class ConfBlock extends HtmlBlock {
     }
     Path confPath = job.getConfFile();
     try {
-      ConfInfo info = new ConfInfo(job, this.conf);
+      ConfInfo info = new ConfInfo(job);
 
       html.div().h3(confPath.toString())._();
       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().
             td(".id", taid).
             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) {
           String containerIdStr = ta.getAssignedContainerIdStr();
           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(Job job, Configuration conf) throws IOException {
+  public ConfInfo(Job job) throws IOException {
 
-    Path confPath = job.getConfFile();
     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) {
       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
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-          t.taskID, 0, TaskType.MAP, "", null)));
+          t.taskID, null, 0, TaskType.MAP, "", null)));
       verify(mockWriter).flush();
 
     } finally {
@@ -129,7 +129,7 @@ public class TestJobHistoryEventHandler {
 
       for (int i = 0 ; i < 100 ; i++) {
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-            t.taskID, 0, TaskType.MAP, "", null)));
+            t.taskID, null, 0, TaskType.MAP, "", null)));
       }
 
       handleNextNEvents(jheh, 9);
@@ -174,7 +174,7 @@ public class TestJobHistoryEventHandler {
 
       for (int i = 0 ; i < 100 ; i++) {
         queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
-            t.taskID, 0, TaskType.MAP, "", null)));
+            t.taskID, null, 0, TaskType.MAP, "", null)));
       }
 
       handleNextNEvents(jheh, 9);
@@ -215,7 +215,7 @@ public class TestJobHistoryEventHandler {
 
       for (int i = 0 ; i < 100 ; i++) {
         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(
           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
   protected ContainerAllocator createContainerAllocator(
       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) {
         ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
         cId.setApplicationAttemptId(getContext().getApplicationAttemptId());
@@ -452,7 +456,6 @@ public class MRApp extends MRAppMaster {
             new TaskAttemptContainerAssignedEvent(event.getAttemptID(),
                 container, null));
       }
-    };
   }
 
   @Override

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