Ver código fonte

Merge r1327258 through r1329943 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3092@1329947 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 anos atrás
pai
commit
18a89accf8
100 arquivos alterados com 1700 adições e 863 exclusões
  1. 0 16
      hadoop-client/pom.xml
  2. 1 1
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
  3. 1 1
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java
  4. 19 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  5. 5 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  6. 23 15
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java
  7. 36 33
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  8. 5 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  9. 3 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoServerSideTranslatorPB.java
  10. 15 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
  11. 1 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java
  12. 0 118
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
  13. 27 23
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  14. 7 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  15. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
  16. 30 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  17. 28 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
  18. 0 58
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  19. 0 5
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml
  20. 58 0
      hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto
  21. 20 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.io.compress.CompressionCodec
  22. 5 3
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  23. 0 4
      hadoop-common-project/hadoop-common/src/site/apt/DeprecatedProperties.apt.vm
  24. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodec.java
  25. 33 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java
  26. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
  27. 1 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
  28. 1 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
  29. 4 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
  30. 1 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
  31. 9 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
  32. 2 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  33. 3 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
  34. 16 9
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
  35. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccess.java
  36. 5 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/FileSystemAccessException.java
  37. 52 35
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
  38. 0 31
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
  39. 8 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
  40. 17 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java
  42. 70 11
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
  43. 71 6
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java
  44. 6 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HadoopUsersConfTestHelper.java
  45. 35 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  46. 6 34
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  48. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  49. 4 45
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  50. 19 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  51. 34 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  52. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  53. 71 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
  54. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  55. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  56. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  57. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
  58. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  59. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
  60. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
  61. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
  62. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
  63. 20 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
  64. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  65. 47 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
  66. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  67. 27 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  68. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  69. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
  70. 96 37
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  71. 30 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  72. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  73. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  74. 151 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpDelegationToken.java
  75. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
  76. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
  77. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  78. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java
  79. 42 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
  80. 101 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
  81. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  82. 39 31
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  83. 53 0
      hadoop-mapreduce-project/CHANGES.txt
  84. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
  85. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
  86. 18 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  87. 6 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  88. 3 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  89. 2 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
  90. 14 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  91. 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
  92. 11 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
  93. 2 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  94. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  95. 71 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  96. 26 30
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
  97. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java
  98. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
  99. 16 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  100. 13 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/AbstractCounters.java

+ 0 - 16
hadoop-client/pom.xml

@@ -171,10 +171,6 @@
           <groupId>junit</groupId>
           <artifactId>junit</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>com.cenqua.clover</groupId>
-          <artifactId>clover</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>org.apache.avro</groupId>
           <artifactId>avro</artifactId>
@@ -211,10 +207,6 @@
           <groupId>com.sun.jersey.contribs</groupId>
           <artifactId>jersey-guice</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>com.cenqua.clover</groupId>
-          <artifactId>clover</artifactId>
-        </exclusion>
         <exclusion>
           <groupId>com.google.inject.extensions</groupId>
           <artifactId>guice-servlet</artifactId>
@@ -263,10 +255,6 @@
           <groupId>com.google.inject.extensions</groupId>
           <artifactId>guice-servlet</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>com.cenqua.clover</groupId>
-          <artifactId>clover</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
 
@@ -291,10 +279,6 @@
           <groupId>com.google.inject.extensions</groupId>
           <artifactId>guice-servlet</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>com.cenqua.clover</groupId>
-          <artifactId>clover</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
   </dependencies>

+ 1 - 1
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java

@@ -288,7 +288,7 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
                 String clientPrincipal = gssContext.getSrcName().toString();
                 KerberosName kerberosName = new KerberosName(clientPrincipal);
                 String userName = kerberosName.getShortName();
-                token = new AuthenticationToken(userName, clientPrincipal, TYPE);
+                token = new AuthenticationToken(userName, clientPrincipal, getType());
                 response.setStatus(HttpServletResponse.SC_OK);
                 LOG.trace("SPNEGO completed for principal [{}]", clientPrincipal);
               }

+ 1 - 1
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/PseudoAuthenticationHandler.java

@@ -126,7 +126,7 @@ public class PseudoAuthenticationHandler implements AuthenticationHandler {
         throw new AuthenticationException("Anonymous requests are disallowed");
       }
     } else {
-      token = new AuthenticationToken(userName, userName, TYPE);
+      token = new AuthenticationToken(userName, userName, getType());
     }
     return token;
   }

+ 19 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -61,7 +61,9 @@ Trunk (unreleased changes)
     HADOOP-8147. test-patch should run tests with -fn to avoid masking test
     failures (Robert Evans via tgraves)
 
-    HADOOP-8117. Upgrade test build to Surefire 2.12 (todd)
+    HADOOP-8290. Remove remaining references to hadoop.native.lib (harsh)
+
+    HADOOP-8285 Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
 
   BUG FIXES
 
@@ -263,6 +265,10 @@ Release 2.0.0 - UNRELEASED
     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)
+
+    HADOOP-8152. Expand public APIs for security library classes. (atm via eli)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -358,6 +364,15 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8282. start-all.sh refers incorrectly start-dfs.sh
     existence for starting start-yarn.sh. (Devaraj K via eli)
 
+    HADOOP-7350. Use ServiceLoader to discover compression codec classes.
+    (tomwhite)
+
+    HADOOP-8284. clover integration broken, also mapreduce poms are pulling
+    in clover as a dependency. (phunt via tucu)
+
+    HADOOP-8309. Pseudo & Kerberos AuthenticationHandler should use 
+    getType() to create token (tucu)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -421,6 +436,9 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8108. Move method getHostPortString() from NameNode to NetUtils.
     (Brandon Li via jitendra)
 
+    HADOOP-8288. Remove references of mapred.child.ulimit etc. since they are
+    not being used any more (Ravi Prakash via bobby)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 5 - 0
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -281,9 +281,14 @@
     <Match>
       <!-- protobuf generated code -->
       <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.IpcConnectionContextProtos.*"/>
+    </Match>
+        <Match>
+      <!-- protobuf generated code -->
+      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.RpcPayloadHeaderProtos.*"/>
     </Match>
     <Match>
       <!-- protobuf generated code -->
       <Class name="~org\.apache\.hadoop\.ha\.proto\.HAServiceProtocolProtos.*"/>
     </Match>
+
  </FindBugsFilter>

+ 23 - 15
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java

@@ -36,6 +36,9 @@ public class CompressionCodecFactory {
 
   public static final Log LOG =
     LogFactory.getLog(CompressionCodecFactory.class.getName());
+  
+  private static final ServiceLoader<CompressionCodec> CODEC_PROVIDERS =
+    ServiceLoader.load(CompressionCodec.class);
 
   /**
    * A map from the reversed filename suffixes to the codecs.
@@ -95,16 +98,23 @@ public class CompressionCodecFactory {
   }
 
   /**
-   * Get the list of codecs listed in the configuration
+   * Get the list of codecs discovered via a Java ServiceLoader, or
+   * listed in the configuration. Codecs specified in configuration come
+   * later in the returned list, and are considered to override those
+   * from the ServiceLoader.
    * @param conf the configuration to look in
-   * @return a list of the Configuration classes or null if the attribute
-   *         was not set
+   * @return a list of the {@link CompressionCodec} classes
    */
   public static List<Class<? extends CompressionCodec>> getCodecClasses(Configuration conf) {
+    List<Class<? extends CompressionCodec>> result
+      = new ArrayList<Class<? extends CompressionCodec>>();
+    // Add codec classes discovered via service loading
+    for (CompressionCodec codec : CODEC_PROVIDERS) {
+      result.add(codec.getClass());
+    }
+    // Add codec classes from configuration
     String codecsString = conf.get("io.compression.codecs");
     if (codecsString != null) {
-      List<Class<? extends CompressionCodec>> result
-        = new ArrayList<Class<? extends CompressionCodec>>();
       StringTokenizer codecSplit = new StringTokenizer(codecsString, ",");
       while (codecSplit.hasMoreElements()) {
         String codecSubstring = codecSplit.nextToken();
@@ -123,14 +133,14 @@ public class CompressionCodecFactory {
           }
         }
       }
-      return result;
-    } else {
-      return null;
     }
+    return result;
   }
   
   /**
-   * Sets a list of codec classes in the configuration.
+   * Sets a list of codec classes in the configuration. In addition to any
+   * classes specified using this method, {@link CompressionCodec} classes on
+   * the classpath are discovered using a Java ServiceLoader.
    * @param conf the configuration to modify
    * @param classes the list of classes to set
    */
@@ -151,21 +161,19 @@ public class CompressionCodecFactory {
   
   /**
    * Find the codecs specified in the config value io.compression.codecs 
-   * and register them. Defaults to gzip and zip.
+   * and register them. Defaults to gzip and deflate.
    */
   public CompressionCodecFactory(Configuration conf) {
     codecs = new TreeMap<String, CompressionCodec>();
     codecsByClassName = new HashMap<String, CompressionCodec>();
     codecsByName = new HashMap<String, CompressionCodec>();
     List<Class<? extends CompressionCodec>> codecClasses = getCodecClasses(conf);
-    if (codecClasses == null) {
+    if (codecClasses == null || codecClasses.isEmpty()) {
       addCodec(new GzipCodec());
       addCodec(new DefaultCodec());      
     } else {
-      Iterator<Class<? extends CompressionCodec>> itr = codecClasses.iterator();
-      while (itr.hasNext()) {
-        CompressionCodec codec = ReflectionUtils.newInstance(itr.next(), conf);
-        addCodec(codec);     
+      for (Class<? extends CompressionCodec> codecClass : codecClasses) {
+        addCodec(ReflectionUtils.newInstance(codecClass, conf));
       }
     }
   }

+ 36 - 33
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -50,8 +50,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.ipc.RpcPayloadHeader.*;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
+import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcPayloadHeaderProto;
+import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcPayloadOperationProto;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
@@ -163,10 +164,10 @@ public class Client {
     final Writable rpcRequest;  // the serialized rpc request - RpcPayload
     Writable rpcResponse;       // null if rpc has error
     IOException error;          // exception, null if success
-    final RpcKind rpcKind;      // Rpc EngineKind
+    final RPC.RpcKind rpcKind;      // Rpc EngineKind
     boolean done;               // true when call is done
 
-    protected Call(RpcKind rpcKind, Writable param) {
+    protected Call(RPC.RpcKind rpcKind, Writable param) {
       this.rpcKind = rpcKind;
       this.rpcRequest = param;
       synchronized (Client.this) {
@@ -613,7 +614,7 @@ public class Client {
             this.in = new DataInputStream(new BufferedInputStream(inStream));
           }
           this.out = new DataOutputStream(new BufferedOutputStream(outStream));
-          writeHeader();
+          writeConnectionContext();
 
           // update last activity time
           touch();
@@ -704,16 +705,17 @@ public class Client {
       out.flush();
     }
     
-    /* Write the protocol header for each connection
+    /* Write the connection context header for each connection
      * Out is not synchronized because only the first thread does this.
      */
-    private void writeHeader() throws IOException {
+    private void writeConnectionContext() throws IOException {
       // Write out the ConnectionHeader
       DataOutputBuffer buf = new DataOutputBuffer();
       connectionContext.writeTo(buf);
       
       // Write out the payload length
       int bufLen = buf.getLength();
+
       out.writeInt(bufLen);
       out.write(buf.getData(), 0, bufLen);
     }
@@ -806,21 +808,22 @@ public class Client {
           if (LOG.isDebugEnabled())
             LOG.debug(getName() + " sending #" + call.id);
           
-          //for serializing the
-          //data to be written
+          // Serializing the data to be written.
+          // Format:
+          // 0) Length of rest below (1 + 2)
+          // 1) PayloadHeader  - is serialized Delimited hence contains length
+          // 2) the Payload - the RpcRequest
+          //
           d = new DataOutputBuffer();
-          d.writeInt(0); // placeholder for data length
-          RpcPayloadHeader header = new RpcPayloadHeader(
-              call.rpcKind, RpcPayloadOperation.RPC_FINAL_PAYLOAD, call.id);
-          header.write(d);
+          RpcPayloadHeaderProto header = ProtoUtil.makeRpcPayloadHeader(
+             call.rpcKind, RpcPayloadOperationProto.RPC_FINAL_PAYLOAD, call.id);
+          header.writeDelimitedTo(d);
           call.rpcRequest.write(d);
           byte[] data = d.getData();
-          int dataLength = d.getLength() - 4;
-          data[0] = (byte)((dataLength >>> 24) & 0xff);
-          data[1] = (byte)((dataLength >>> 16) & 0xff);
-          data[2] = (byte)((dataLength >>> 8) & 0xff);
-          data[3] = (byte)(dataLength & 0xff);
-          out.write(data, 0, dataLength + 4);//write the data
+   
+          int totalLength = d.getLength();
+          out.writeInt(totalLength); // Total Length
+          out.write(data, 0, totalLength);//PayloadHeader + RpcRequest
           out.flush();
         }
       } catch(IOException e) {
@@ -937,7 +940,7 @@ public class Client {
     private int index;
     
     public ParallelCall(Writable param, ParallelResults results, int index) {
-      super(RpcKind.RPC_WRITABLE, param);
+      super(RPC.RpcKind.RPC_WRITABLE, param);
       this.results = results;
       this.index = index;
     }
@@ -1022,22 +1025,22 @@ public class Client {
   }
 
   /**
-   * Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
+   * Same as {@link #call(RPC.RpcKind, Writable, ConnectionId)}
    *  for RPC_BUILTIN
    */
   public Writable call(Writable param, InetSocketAddress address)
   throws InterruptedException, IOException {
-    return call(RpcKind.RPC_BUILTIN, param, address);
+    return call(RPC.RpcKind.RPC_BUILTIN, param, address);
     
   }
   /** Make a call, passing <code>param</code>, to the IPC server running at
    * <code>address</code>, returning the value.  Throws exceptions if there are
    * network problems or if the remote code threw an exception.
-   * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+   * @deprecated Use {@link #call(RPC.RpcKind, Writable,
    *  ConnectionId)} instead 
    */
   @Deprecated
-  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress address)
+  public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress address)
   throws InterruptedException, IOException {
       return call(rpcKind, param, address, null);
   }
@@ -1047,11 +1050,11 @@ public class Client {
    * the value.  
    * Throws exceptions if there are network problems or if the remote code 
    * threw an exception.
-   * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable, 
+   * @deprecated Use {@link #call(RPC.RpcKind, Writable, 
    * ConnectionId)} instead 
    */
   @Deprecated
-  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, 
+  public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr, 
       UserGroupInformation ticket)  
       throws InterruptedException, IOException {
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, null, ticket, 0,
@@ -1065,11 +1068,11 @@ public class Client {
    * timeout, returning the value.  
    * Throws exceptions if there are network problems or if the remote code 
    * threw an exception. 
-   * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+   * @deprecated Use {@link #call(RPC.RpcKind, Writable,
    *  ConnectionId)} instead 
    */
   @Deprecated
-  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, 
+  public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr, 
                        Class<?> protocol, UserGroupInformation ticket,
                        int rpcTimeout)  
                        throws InterruptedException, IOException {
@@ -1080,7 +1083,7 @@ public class Client {
 
   
   /**
-   * Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, InetSocketAddress, 
+   * Same as {@link #call(RPC.RpcKind, Writable, InetSocketAddress, 
    * Class, UserGroupInformation, int, Configuration)}
    * except that rpcKind is writable.
    */
@@ -1090,7 +1093,7 @@ public class Client {
       throws InterruptedException, IOException {
         ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
         ticket, rpcTimeout, conf);
-    return call(RpcKind.RPC_BUILTIN, param, remoteId);
+    return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
   }
   
   /**
@@ -1101,7 +1104,7 @@ public class Client {
    * value. Throws exceptions if there are network problems or if the remote
    * code threw an exception.
    */
-  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, 
+  public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr, 
                        Class<?> protocol, UserGroupInformation ticket,
                        int rpcTimeout, Configuration conf)  
                        throws InterruptedException, IOException {
@@ -1111,12 +1114,12 @@ public class Client {
   }
   
   /**
-   * Same as {link {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
+   * Same as {link {@link #call(RPC.RpcKind, Writable, ConnectionId)}
    * except the rpcKind is RPC_BUILTIN
    */
   public Writable call(Writable param, ConnectionId remoteId)  
       throws InterruptedException, IOException {
-     return call(RpcKind.RPC_BUILTIN, param, remoteId);
+     return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
   }
   
   /** 
@@ -1130,7 +1133,7 @@ public class Client {
    * Throws exceptions if there are network problems or if the remote code 
    * threw an exception.
    */
-  public Writable call(RpcKind rpcKind, Writable rpcRequest,
+  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
       ConnectionId remoteId) throws InterruptedException, IOException {
     Call call = new Call(rpcKind, rpcRequest);
     Connection connection = getConnection(remoteId, call);

+ 5 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.io.DataOutputOutputStream;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.RPC.RpcInvoker;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 
 import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -61,7 +60,7 @@ public class ProtobufRpcEngine implements RpcEngine {
   
   static { // Register the rpcRequest deserializer for WritableRpcEngine 
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
-        RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWritable.class,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWritable.class,
         new Server.ProtoBufRpcInvoker());
   }
 
@@ -182,7 +181,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args);
       RpcResponseWritable val = null;
       try {
-        val = (RpcResponseWritable) client.call(RpcKind.RPC_PROTOCOL_BUFFER,
+        val = (RpcResponseWritable) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
             new RpcRequestWritable(rpcRequest), remoteId);
       } catch (Throwable e) {
         throw new ServiceException(e);
@@ -351,7 +350,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
               .getClass().getName()), secretManager, portRangeConfig);
       this.verbose = verbose;  
-      registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
+      registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
           protocolImpl);
     }
     
@@ -363,10 +362,10 @@ public class ProtobufRpcEngine implements RpcEngine {
           String protoName, long version) throws IOException {
         ProtoNameVer pv = new ProtoNameVer(protoName, version);
         ProtoClassProtoImpl impl = 
-            server.getProtocolImplMap(RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
+            server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
         if (impl == null) { // no match for Protocol AND Version
           VerProtocolImpl highest = 
-              server.getHighestSupportedProtocol(RpcKind.RPC_PROTOCOL_BUFFER, 
+              server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, 
                   protoName);
           if (highest == null) {
             throw new IOException("Unknown protocol: " + protoName);

+ 3 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoServerSideTranslatorPB.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.ipc;
 
 import org.apache.hadoop.ipc.RPC.Server.VerProtocolImpl;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolVersionsRequestProto;
@@ -49,7 +48,7 @@ public class ProtocolMetaInfoServerSideTranslatorPB implements
     String protocol = request.getProtocol();
     GetProtocolVersionsResponseProto.Builder builder = 
         GetProtocolVersionsResponseProto.newBuilder();
-    for (RpcKind r : RpcKind.values()) {
+    for (RPC.RpcKind r : RPC.RpcKind.values()) {
       long[] versions;
       try {
         versions = getProtocolVersionForRpcKind(r, protocol);
@@ -78,7 +77,7 @@ public class ProtocolMetaInfoServerSideTranslatorPB implements
     String rpcKind = request.getRpcKind();
     long[] versions;
     try {
-      versions = getProtocolVersionForRpcKind(RpcKind.valueOf(rpcKind),
+      versions = getProtocolVersionForRpcKind(RPC.RpcKind.valueOf(rpcKind),
           protocol);
     } catch (ClassNotFoundException e1) {
       throw new ServiceException(e1);
@@ -104,7 +103,7 @@ public class ProtocolMetaInfoServerSideTranslatorPB implements
     return builder.build();
   }
   
-  private long[] getProtocolVersionForRpcKind(RpcKind rpcKind,
+  private long[] getProtocolVersionForRpcKind(RPC.RpcKind rpcKind,
       String protocol) throws ClassNotFoundException {
     Class<?> protocolClass = Class.forName(protocol);
     String protocolName = RPC.getProtocolName(protocolClass);

+ 15 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java

@@ -42,7 +42,6 @@ import org.apache.commons.logging.*;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.Client.ConnectionId;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SaslRpcServer;
@@ -73,6 +72,18 @@ import com.google.protobuf.BlockingService;
  * the protocol instance is transmitted.
  */
 public class RPC {
+  public enum RpcKind {
+    RPC_BUILTIN ((short) 1),         // Used for built in calls by tests
+    RPC_WRITABLE ((short) 2),        // Use WritableRpcEngine 
+    RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
+    final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
+    private static final short FIRST_INDEX = RPC_BUILTIN.value;    
+    public final short value; //TODO make it private
+
+    RpcKind(short val) {
+      this.value = val;
+    } 
+  }
   
   interface RpcInvoker {   
     /**
@@ -777,7 +788,7 @@ public class RPC {
    ArrayList<Map<ProtoNameVer, ProtoClassProtoImpl>> protocolImplMapArray = 
        new ArrayList<Map<ProtoNameVer, ProtoClassProtoImpl>>(RpcKind.MAX_INDEX);
    
-   Map<ProtoNameVer, ProtoClassProtoImpl> getProtocolImplMap(RpcKind rpcKind) {
+   Map<ProtoNameVer, ProtoClassProtoImpl> getProtocolImplMap(RPC.RpcKind rpcKind) {
      if (protocolImplMapArray.size() == 0) {// initialize for all rpc kinds
        for (int i=0; i <= RpcKind.MAX_INDEX; ++i) {
          protocolImplMapArray.add(
@@ -821,7 +832,7 @@ public class RPC {
    
    
    @SuppressWarnings("unused") // will be useful later.
-   VerProtocolImpl[] getSupportedProtocolVersions(RpcKind rpcKind,
+   VerProtocolImpl[] getSupportedProtocolVersions(RPC.RpcKind rpcKind,
        String protocolName) {
      VerProtocolImpl[] resultk = 
          new  VerProtocolImpl[getProtocolImplMap(rpcKind).size()];
@@ -900,7 +911,7 @@ public class RPC {
     }
     
     @Override
-    public Writable call(RpcKind rpcKind, String protocol,
+    public Writable call(RPC.RpcKind rpcKind, String protocol,
         Writable rpcRequest, long receiveTime) throws Exception {
       return getRpcInvoker(rpcKind).call(this, protocol, rpcRequest,
           receiveTime);

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

@@ -27,7 +27,6 @@ import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
@@ -107,7 +106,7 @@ public class RpcClientUtil {
    * @throws IOException
    */
   public static boolean isMethodSupported(Object rpcProxy, Class<?> protocol,
-      RpcKind rpcKind, long version, String methodName) throws IOException {
+      RPC.RpcKind rpcKind, long version, String methodName) throws IOException {
     InetSocketAddress serverAddress = RPC.getServerAddress(rpcProxy);
     Map<Long, ProtocolSignature> versionMap = getVersionSignatureMap(
         serverAddress, protocol.getName(), rpcKind.toString());

+ 0 - 118
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java

@@ -1,118 +0,0 @@
-package org.apache.hadoop.ipc;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * This is the rpc payload header. It is sent with every rpc call
- * <pre>
- * The format of RPC call is as follows:
- * +---------------------------------------------------+
- * |  Rpc length in bytes (header + payload length)    |
- * +---------------------------------------------------+
- * |      Rpc Header       |       Rpc Payload         |
- * +---------------------------------------------------+
- * 
- * The format of Rpc Header is:
- * +----------------------------------+
- * |  RpcKind (1 bytes)               |      
- * +----------------------------------+
- * |  RpcPayloadOperation (1 bytes)   |      
- * +----------------------------------+
- * |  Call ID (4 bytes)               |      
- * +----------------------------------+
- * 
- * {@link RpcKind} determines the type of serialization used for Rpc Payload.
- * </pre>
- * <p>
- * <b>Note this header does NOT have its own version number, 
- * it used the version number from the connection header. </b>
- */
-public class RpcPayloadHeader implements Writable {
-  public enum RpcPayloadOperation {
-    RPC_FINAL_PAYLOAD ((short)1),
-    RPC_CONTINUATION_PAYLOAD ((short)2), // not implemented yet
-    RPC_CLOSE_CONNECTION ((short)3);     // close the rpc connection
-    
-    private final short code;
-    private static final short FIRST_INDEX = RPC_FINAL_PAYLOAD.code;
-    RpcPayloadOperation(short val) {
-      this.code = val;
-    }
-    
-    public void write(DataOutput out) throws IOException {  
-      out.writeByte(code);
-    }
-    
-    static RpcPayloadOperation readFields(DataInput in) throws IOException {
-      short inValue = in.readByte();
-      return RpcPayloadOperation.values()[inValue - FIRST_INDEX];
-    }
-  }
-  
-  public enum RpcKind {
-    RPC_BUILTIN ((short) 1),         // Used for built in calls by tests
-    RPC_WRITABLE ((short) 2),        // Use WritableRpcEngine 
-    RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
-    final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
-    private static final short FIRST_INDEX = RPC_BUILTIN.value;    
-    private final short value;
-
-    RpcKind(short val) {
-      this.value = val;
-    }
-    
-    public void write(DataOutput out) throws IOException {
-      out.writeByte(value);
-    }
-    
-    static RpcKind readFields(DataInput in) throws IOException {
-      short inValue = in.readByte();
-      return RpcKind.values()[inValue - FIRST_INDEX];
-    }  
-  }
-  
-  private RpcKind kind;
-  private RpcPayloadOperation operation;
-  private int callId;
-  
-  public RpcPayloadHeader() {
-    kind = RpcKind.RPC_WRITABLE;
-    operation = RpcPayloadOperation.RPC_CLOSE_CONNECTION;
-  }
-  
-  public RpcPayloadHeader(RpcKind kind, RpcPayloadOperation op, int callId) {
-    this.kind  = kind;
-    this.operation = op;
-    this.callId = callId;
-  }
-  
-  int getCallId() {
-    return callId;
-  }
-  
-  RpcKind getkind() {
-    return kind;
-  }
-  
-  RpcPayloadOperation getOperation() {
-    return operation;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    kind.write(out);
-    operation.write(out);
-    out.writeInt(callId); 
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    kind = RpcKind.readFields(in);
-    operation = RpcPayloadOperation.readFields(in);
-    this.callId = in.readInt();
-  }
-}

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

@@ -72,11 +72,10 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.ipc.RPC.RpcInvoker;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation;
 import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
 import org.apache.hadoop.ipc.metrics.RpcMetrics;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
+import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SaslRpcServer;
@@ -170,8 +169,8 @@ public abstract class Server {
       this.rpcRequestWrapperClass = rpcRequestWrapperClass;
     }   
   }
-  static Map<RpcKind, RpcKindMapValue> rpcKindMap = new
-      HashMap<RpcKind, RpcKindMapValue>(4);
+  static Map<RPC.RpcKind, RpcKindMapValue> rpcKindMap = new
+      HashMap<RPC.RpcKind, RpcKindMapValue>(4);
   
   
 
@@ -185,7 +184,7 @@ public abstract class Server {
    *  @param rpcInvoker - use to process the calls on SS.
    */
   
-  public static void registerProtocolEngine(RpcKind rpcKind, 
+  public static void registerProtocolEngine(RPC.RpcKind rpcKind, 
           Class<? extends Writable> rpcRequestWrapperClass,
           RpcInvoker rpcInvoker) {
     RpcKindMapValue  old = 
@@ -201,14 +200,14 @@ public abstract class Server {
   }
   
   public Class<? extends Writable> getRpcRequestWrapper(
-      RpcKind rpcKind) {
+      RpcKindProto rpcKind) {
     if (rpcRequestClass != null)
        return rpcRequestClass;
-    RpcKindMapValue val = rpcKindMap.get(rpcKind);
+    RpcKindMapValue val = rpcKindMap.get(ProtoUtil.convert(rpcKind));
     return (val == null) ? null : val.rpcRequestWrapperClass; 
   }
   
-  public static RpcInvoker  getRpcInvoker(RpcKind rpcKind) {
+  public static RpcInvoker  getRpcInvoker(RPC.RpcKind rpcKind) {
     RpcKindMapValue val = rpcKindMap.get(rpcKind);
     return (val == null) ? null : val.rpcInvoker; 
   }
@@ -403,12 +402,12 @@ public abstract class Server {
     private long timestamp;               // time received when response is null
                                           // time served when response is not null
     private ByteBuffer rpcResponse;       // the response for this call
-    private final RpcKind rpcKind;
+    private final RPC.RpcKind rpcKind;
 
     public Call(int id, Writable param, Connection connection) {
-      this( id,  param,  connection, RpcKind.RPC_BUILTIN );    
+      this( id,  param,  connection, RPC.RpcKind.RPC_BUILTIN );    
     }
-    public Call(int id, Writable param, Connection connection, RpcKind kind) { 
+    public Call(int id, Writable param, Connection connection, RPC.RpcKind kind) { 
       this.callId = id;
       this.rpcRequest = param;
       this.connection = connection;
@@ -1366,7 +1365,6 @@ public abstract class Server {
         if (data == null) {
           dataLengthBuffer.flip();
           dataLength = dataLengthBuffer.getInt();
-       
           if ((dataLength == Client.PING_CALL_ID) && (!useWrap)) {
             // covers the !useSasl too
             dataLengthBuffer.clear();
@@ -1555,22 +1553,27 @@ public abstract class Server {
     private void processData(byte[] buf) throws  IOException, InterruptedException {
       DataInputStream dis =
         new DataInputStream(new ByteArrayInputStream(buf));
-      RpcPayloadHeader header = new RpcPayloadHeader();
-      header.readFields(dis);           // Read the RpcPayload header
+      RpcPayloadHeaderProto header = RpcPayloadHeaderProto.parseDelimitedFrom(dis);
         
       if (LOG.isDebugEnabled())
         LOG.debug(" got #" + header.getCallId());
-      if (header.getOperation() != RpcPayloadOperation.RPC_FINAL_PAYLOAD) {
+      if (!header.hasRpcOp()) {
+        throw new IOException(" IPC Server: No rpc op in rpcPayloadHeader");
+      }
+      if (header.getRpcOp() != RpcPayloadOperationProto.RPC_FINAL_PAYLOAD) {
         throw new IOException("IPC Server does not implement operation" + 
-              header.getOperation());
+              header.getRpcOp());
       }
       // If we know the rpc kind, get its class so that we can deserialize
       // (Note it would make more sense to have the handler deserialize but 
       // we continue with this original design.
+      if (!header.hasRpcKind()) {
+        throw new IOException(" IPC Server: No rpc kind in rpcPayloadHeader");
+      }
       Class<? extends Writable> rpcRequestClass = 
-          getRpcRequestWrapper(header.getkind());
+          getRpcRequestWrapper(header.getRpcKind());
       if (rpcRequestClass == null) {
-        LOG.warn("Unknown rpc kind "  + header.getkind() + 
+        LOG.warn("Unknown rpc kind "  + header.getRpcKind() + 
             " from client " + getHostAddress());
         final Call readParamsFailedCall = 
             new Call(header.getCallId(), null, this);
@@ -1578,7 +1581,7 @@ public abstract class Server {
 
         setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
             IOException.class.getName(),
-            "Unknown rpc kind "  + header.getkind());
+            "Unknown rpc kind "  + header.getRpcKind());
         responder.doRespond(readParamsFailedCall);
         return;   
       }
@@ -1589,7 +1592,7 @@ public abstract class Server {
       } catch (Throwable t) {
         LOG.warn("Unable to read call parameters for client " +
                  getHostAddress() + "on connection protocol " +
-            this.protocolName + " for rpcKind " + header.getkind(),  t);
+            this.protocolName + " for rpcKind " + header.getRpcKind(),  t);
         final Call readParamsFailedCall = 
             new Call(header.getCallId(), null, this);
         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
@@ -1601,7 +1604,8 @@ public abstract class Server {
         return;
       }
         
-      Call call = new Call(header.getCallId(), rpcRequest, this, header.getkind());
+      Call call = new Call(header.getCallId(), rpcRequest, this, 
+          ProtoUtil.convert(header.getRpcKind()));
       callQueue.put(call);              // queue the call; maybe blocked here
       incRpcCount();  // Increment the rpc count
     }
@@ -1991,11 +1995,11 @@ public abstract class Server {
    */
   @Deprecated
   public Writable call(Writable param, long receiveTime) throws Exception {
-    return call(RpcKind.RPC_BUILTIN, null, param, receiveTime);
+    return call(RPC.RpcKind.RPC_BUILTIN, null, param, receiveTime);
   }
   
   /** Called for each call. */
-  public abstract Writable call(RpcKind rpcKind, String protocol,
+  public abstract Writable call(RPC.RpcKind rpcKind, String protocol,
       Writable param, long receiveTime) throws Exception;
   
   /**

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

@@ -33,7 +33,6 @@ import org.apache.commons.logging.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.RPC.RpcInvoker;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
@@ -75,7 +74,7 @@ public class WritableRpcEngine implements RpcEngine {
    * Register the rpcRequest deserializer for WritableRpcEngine
    */
   private static synchronized void initialize() {
-    org.apache.hadoop.ipc.Server.registerProtocolEngine(RpcKind.RPC_WRITABLE,
+    org.apache.hadoop.ipc.Server.registerProtocolEngine(RPC.RpcKind.RPC_WRITABLE,
         Invocation.class, new Server.WritableRpcInvoker());
     isInitialized = true;
   }
@@ -223,7 +222,7 @@ public class WritableRpcEngine implements RpcEngine {
       }
 
       ObjectWritable value = (ObjectWritable)
-        client.call(RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
+        client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
       if (LOG.isDebugEnabled()) {
         long callTime = System.currentTimeMillis() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
@@ -412,12 +411,12 @@ public class WritableRpcEngine implements RpcEngine {
               protocolImpl.getClass());
         }
         // register protocol class and its super interfaces
-        registerProtocolAndImpl(RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
+        registerProtocolAndImpl(RPC.RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
         protocols = RPC.getProtocolInterfaces(protocolClass);
       }
       for (Class<?> p : protocols) {
         if (!p.equals(VersionedProtocol.class)) {
-          registerProtocolAndImpl(RpcKind.RPC_WRITABLE, p, protocolImpl);
+          registerProtocolAndImpl(RPC.RpcKind.RPC_WRITABLE, p, protocolImpl);
         }
       }
 
@@ -461,7 +460,7 @@ public class WritableRpcEngine implements RpcEngine {
             // registered directly.
             // Send the call to the highest  protocol version
             VerProtocolImpl highest = server.getHighestSupportedProtocol(
-                RpcKind.RPC_WRITABLE, protocolName);
+                RPC.RpcKind.RPC_WRITABLE, protocolName);
             if (highest == null) {
               throw new IOException("Unknown protocol: " + protocolName);
             }
@@ -473,10 +472,10 @@ public class WritableRpcEngine implements RpcEngine {
             ProtoNameVer pv = 
                 new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
             protocolImpl = 
-                server.getProtocolImplMap(RpcKind.RPC_WRITABLE).get(pv);
+                server.getProtocolImplMap(RPC.RpcKind.RPC_WRITABLE).get(pv);
             if (protocolImpl == null) { // no match for Protocol AND Version
                VerProtocolImpl highest = 
-                   server.getHighestSupportedProtocol(RpcKind.RPC_WRITABLE, 
+                   server.getHighestSupportedProtocol(RPC.RpcKind.RPC_WRITABLE, 
                        protoName);
               if (highest == null) {
                 throw new IOException("Unknown protocol: " + protoName);

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java

@@ -220,6 +220,8 @@ public class SecurityUtil {
    * @return converted Kerberos principal name
    * @throws IOException if the client address cannot be determined
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static String getServerPrincipal(String principalConfig,
       String hostname) throws IOException {
     String[] components = getComponents(principalConfig);
@@ -245,6 +247,8 @@ public class SecurityUtil {
    * @return converted Kerberos principal name
    * @throws IOException if the client address cannot be determined
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static String getServerPrincipal(String principalConfig,
       InetAddress addr) throws IOException {
     String[] components = getComponents(principalConfig);
@@ -292,6 +296,8 @@ public class SecurityUtil {
    *          the key to look for user's Kerberos principal name in conf
    * @throws IOException if login fails
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static void login(final Configuration conf,
       final String keytabFileKey, final String userNameKey) throws IOException {
     login(conf, keytabFileKey, userNameKey, getLocalHostName());
@@ -312,6 +318,8 @@ public class SecurityUtil {
    *          hostname to use for substitution
    * @throws IOException if the config doesn't specify a keytab
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static void login(final Configuration conf,
       final String keytabFileKey, final String userNameKey, String hostname)
       throws IOException {

+ 30 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -69,7 +69,7 @@ import org.apache.hadoop.util.Shell;
  * user's username and groups. It supports both the Windows, Unix and Kerberos 
  * login modules.
  */
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "HBase", "Hive", "Oozie"})
 @InterfaceStability.Evolving
 public class UserGroupInformation {
   private static final Log LOG =  LogFactory.getLog(UserGroupInformation.class);
@@ -258,6 +258,8 @@ public class UserGroupInformation {
    * group look up service.
    * @param conf the configuration to use
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static void setConfiguration(Configuration conf) {
     initialize(conf, false);
   }
@@ -500,6 +502,8 @@ public class UserGroupInformation {
    * @return the current user
    * @throws IOException if login fails
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public synchronized
   static UserGroupInformation getCurrentUser() throws IOException {
     AccessControlContext context = AccessController.getContext();
@@ -516,6 +520,8 @@ public class UserGroupInformation {
    * @return the logged in user
    * @throws IOException if login fails
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public synchronized 
   static UserGroupInformation getLoginUser() throws IOException {
     if (loginUser == null) {
@@ -652,6 +658,8 @@ public class UserGroupInformation {
    * @param path the path to the keytab file
    * @throws IOException if the keytab file can't be read
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public synchronized
   static void loginUserFromKeytab(String user,
                                   String path
@@ -710,6 +718,8 @@ public class UserGroupInformation {
    * the new credentials.
    * @throws IOException on a failure
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public synchronized void reloginFromKeytab()
   throws IOException {
     if (!isSecurityEnabled() ||
@@ -769,6 +779,8 @@ public class UserGroupInformation {
    * the new credentials.
    * @throws IOException on a failure
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public synchronized void reloginFromTicketCache()
   throws IOException {
     if (!isSecurityEnabled() || 
@@ -867,6 +879,8 @@ public class UserGroupInformation {
    * Did the login happen via keytab
    * @return true or false
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public synchronized static boolean isLoginKeytabBased() throws IOException {
     return getLoginUser().isKeytab;
   }
@@ -877,6 +891,8 @@ public class UserGroupInformation {
    * @param user the full user principal name, must not be empty or null
    * @return the UserGroupInformation for the remote user.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static UserGroupInformation createRemoteUser(String user) {
     if (user == null || "".equals(user)) {
       throw new IllegalArgumentException("Null user");
@@ -891,6 +907,7 @@ public class UserGroupInformation {
   /**
    * existing types of authentications' methods
    */
+  @InterfaceAudience.Public
   @InterfaceStability.Evolving
   public static enum AuthenticationMethod {
     SIMPLE,
@@ -908,6 +925,8 @@ public class UserGroupInformation {
    * @param realUser
    * @return proxyUser ugi
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static UserGroupInformation createProxyUser(String user,
       UserGroupInformation realUser) {
     if (user == null || "".equals(user)) {
@@ -929,6 +948,8 @@ public class UserGroupInformation {
    * get RealUser (vs. EffectiveUser)
    * @return realUser running over proxy user
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public UserGroupInformation getRealUser() {
     for (RealUser p: subject.getPrincipals(RealUser.class)) {
       return p.getRealUser();
@@ -974,7 +995,8 @@ public class UserGroupInformation {
    * @param userGroups the names of the groups that the user belongs to
    * @return a fake user for running unit tests
    */
-  @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public static UserGroupInformation createUserForTesting(String user, 
                                                           String[] userGroups) {
     ensureInitialized();
@@ -1000,7 +1022,6 @@ public class UserGroupInformation {
    *          the names of the groups that the user belongs to
    * @return a fake user for running unit tests
    */
-  @InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" })
   public static UserGroupInformation createProxyUserForTesting(String user,
       UserGroupInformation realUser, String[] userGroups) {
     ensureInitialized();
@@ -1029,6 +1050,8 @@ public class UserGroupInformation {
    * Get the user's full principal name.
    * @return the user's full principal name.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public String getUserName() {
     return user.getName();
   }
@@ -1182,6 +1205,8 @@ public class UserGroupInformation {
    * @param action the method to execute
    * @return the value from the run method
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public <T> T doAs(PrivilegedAction<T> action) {
     logPrivilegedAction(subject, action);
     return Subject.doAs(subject, action);
@@ -1198,6 +1223,8 @@ public class UserGroupInformation {
    * @throws InterruptedException if the action throws an InterruptedException
    * @throws UndeclaredThrowableException if the action throws something else
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
   public <T> T doAs(PrivilegedExceptionAction<T> action
                     ) throws IOException, InterruptedException {
     try {

+ 28 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java

@@ -21,8 +21,10 @@ package org.apache.hadoop.util;
 import java.io.DataInput;
 import java.io.IOException;
 
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformationProto;
+import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -135,4 +137,30 @@ public abstract class ProtoUtil {
     }
     return ugi;
   }
+  
+  static RpcKindProto convert(RPC.RpcKind kind) {
+    switch (kind) {
+    case RPC_BUILTIN: return RpcKindProto.RPC_BUILTIN;
+    case RPC_WRITABLE: return RpcKindProto.RPC_WRITABLE;
+    case RPC_PROTOCOL_BUFFER: return RpcKindProto.RPC_PROTOCOL_BUFFER;
+    }
+    return null;
+  }
+  
+  
+  public static RPC.RpcKind convert( RpcKindProto kind) {
+    switch (kind) {
+    case RPC_BUILTIN: return RPC.RpcKind.RPC_BUILTIN;
+    case RPC_WRITABLE: return RPC.RpcKind.RPC_WRITABLE;
+    case RPC_PROTOCOL_BUFFER: return RPC.RpcKind.RPC_PROTOCOL_BUFFER;
+    }
+    return null;
+  }
+ 
+  public static RpcPayloadHeaderProto makeRpcPayloadHeader(RPC.RpcKind rpcKind,
+      RpcPayloadOperationProto operation, int callId) {
+    RpcPayloadHeaderProto.Builder result = RpcPayloadHeaderProto.newBuilder();
+    result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId);
+    return result.build();
+  }
 }

+ 0 - 58
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -81,64 +81,6 @@ abstract public class Shell {
   /** If or not script timed out*/
   private AtomicBoolean timedOut;
 
-  /** a Unix command to get ulimit of a process. */
-  public static final String ULIMIT_COMMAND = "ulimit";
-  
-  /** 
-   * Get the Unix command for setting the maximum virtual memory available
-   * to a given child process. This is only relevant when we are forking a
-   * process from within the Mapper or the Reducer implementations.
-   * Also see Hadoop Pipes and Hadoop Streaming.
-   * 
-   * It also checks to ensure that we are running on a *nix platform else 
-   * (e.g. in Cygwin/Windows) it returns <code>null</code>.
-   * @param memoryLimit virtual memory limit
-   * @return a <code>String[]</code> with the ulimit command arguments or 
-   *         <code>null</code> if we are running on a non *nix platform or
-   *         if the limit is unspecified.
-   */
-  public static String[] getUlimitMemoryCommand(int memoryLimit) {
-    // ulimit isn't supported on Windows
-    if (WINDOWS) {
-      return null;
-    }
-    
-    return new String[] {ULIMIT_COMMAND, "-v", String.valueOf(memoryLimit)};
-  }
-  
-  /** 
-   * Get the Unix command for setting the maximum virtual memory available
-   * to a given child process. This is only relevant when we are forking a
-   * process from within the Mapper or the Reducer implementations.
-   * see also Hadoop Pipes and Streaming.
-   * 
-   * It also checks to ensure that we are running on a *nix platform else 
-   * (e.g. in Cygwin/Windows) it returns <code>null</code>.
-   * @param conf configuration
-   * @return a <code>String[]</code> with the ulimit command arguments or 
-   *         <code>null</code> if we are running on a non *nix platform or
-   *         if the limit is unspecified.
-   * @deprecated Use {@link #getUlimitMemoryCommand(int)}
-   */
-  @Deprecated
-  public static String[] getUlimitMemoryCommand(Configuration conf) {
-    // ulimit isn't supported on Windows
-    if (WINDOWS) {
-      return null;
-    }
-    
-    // get the memory limit from the configuration
-    String ulimit = conf.get("mapred.child.ulimit");
-    if (ulimit == null) {
-      return null;
-    }
-    
-    // Parse it to ensure it is legal/sane
-    int memoryLimit = Integer.valueOf(ulimit);
-    
-    return getUlimitMemoryCommand(memoryLimit);
-  }
-  
   /** Set to true on Windows platforms */
   public static final boolean WINDOWS /* borrowed from Path.WINDOWS */
                 = System.getProperty("os.name").startsWith("Windows");

+ 0 - 5
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml

@@ -108,11 +108,6 @@
     <value>-server -Xmx640m -Djava.net.preferIPv4Stack=true</value>
   </property>
 
-  <property>
-    <name>mapred.child.ulimit</name>
-    <value>8388608</value>
-  </property>
-
   <property>
     <name>mapred.job.tracker.persist.jobstatus.active</name>
     <value>true</value>

+ 58 - 0
hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto

@@ -0,0 +1,58 @@
+/**
+ * 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.
+ */
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "RpcPayloadHeaderProtos";
+option java_generate_equals_and_hash = true;
+
+
+/**
+ * This is the rpc payload header. It is sent with every rpc call.
+ * 
+ * The format of RPC call is as follows:
+ * +-----------------------------------------------------+
+ * |  Rpc length in bytes                                |
+ * +-----------------------------------------------------+
+ * | RpcPayloadHeader - serialized delimited ie has len  |
+ * +-----------------------------------------------------+
+ * |  RpcRequest Payload                                 |
+ * +-----------------------------------------------------+
+ *
+ */
+
+
+
+/**
+ * RpcKind determine the rpcEngine and the serialization of the rpc payload
+ */
+enum RpcKindProto {
+  RPC_BUILTIN          = 0;  // Used for built in calls by tests
+  RPC_WRITABLE         = 1;  // Use WritableRpcEngine 
+  RPC_PROTOCOL_BUFFER  = 2;  // Use ProtobufRpcEngine
+}
+
+enum RpcPayloadOperationProto {
+  RPC_FINAL_PAYLOAD        = 0; // The final payload
+  RPC_CONTINUATION_PAYLOAD = 1; // not implemented yet
+  RPC_CLOSE_CONNECTION     = 2; // close the rpc connection
+}
+   
+message RpcPayloadHeaderProto { // the header for the RpcRequest
+  optional RpcKindProto rpcKind = 1;
+  optional RpcPayloadOperationProto rpcOp = 2;
+  optional uint32 callId = 3; // each rpc has a callId that is also used in response
+}

+ 20 - 0
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.io.compress.CompressionCodec

@@ -0,0 +1,20 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.io.compress.BZip2Codec
+org.apache.hadoop.io.compress.DefaultCodec
+org.apache.hadoop.io.compress.DeflateCodec
+org.apache.hadoop.io.compress.GzipCodec
+org.apache.hadoop.io.compress.Lz4Codec
+org.apache.hadoop.io.compress.SnappyCodec
+

+ 5 - 3
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -268,9 +268,11 @@
 
 <property>
   <name>io.compression.codecs</name>
-  <value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec</value>
-  <description>A list of the compression codec classes that can be used 
-               for compression/decompression.</description>
+  <value></value>
+  <description>A comma-separated list of the compression codec classes that can
+  be used for compression/decompression. In addition to any classes specified
+  with this property (which take precedence), codec classes on the classpath
+  are discovered using a Java ServiceLoader.</description>
 </property>
 
 <property>

+ 0 - 4
hadoop-common-project/hadoop-common/src/site/apt/DeprecatedProperties.apt.vm

@@ -314,8 +314,6 @@ Deprecated Properties
 *---+---+
 |mapred.map.child.log.level | mapreduce.map.log.level
 *---+---+
-|mapred.map.child.ulimit | mapreduce.map.ulimit
-*---+---+
 |mapred.map.max.attempts | mapreduce.map.maxattempts
 *---+---+
 |mapred.map.output.compression.codec | mapreduce.map.output.compress.codec
@@ -378,8 +376,6 @@ Deprecated Properties
 *---+---+
 |mapred.reduce.child.log.level | mapreduce.reduce.log.level
 *---+---+
-|mapred.reduce.child.ulimit | mapreduce.reduce.ulimit
-*---+---+
 |mapred.reduce.max.attempts | mapreduce.reduce.maxattempts
 *---+---+
 |mapred.reduce.parallel.copies | mapreduce.reduce.shuffle.parallelcopies

+ 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));
 

+ 33 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecFactory.java

@@ -101,6 +101,12 @@ public class TestCodecFactory extends TestCase {
     }
   }
   
+  private static class NewGzipCodec extends BaseCodec {
+    public String getDefaultExtension() {
+      return ".gz";
+    }
+  }
+  
   /**
    * Returns a factory for a given set of codecs
    * @param classes the codec classes to include
@@ -167,32 +173,43 @@ public class TestCodecFactory extends TestCase {
     checkCodec("default factory for deflate codec", DeflateCodec.class, codec);
 
     factory = setClasses(new Class[0]);
+    // gz, bz2, snappy, lz4 are picked up by service loader, but bar isn't
     codec = factory.getCodec(new Path("/tmp/foo.bar"));
-    assertEquals("empty codec bar codec", null, codec);
+    assertEquals("empty factory bar codec", null, codec);
     codec = factory.getCodecByClassName(BarCodec.class.getCanonicalName());
-    assertEquals("empty codec bar codec", null, codec);
+    assertEquals("empty factory bar codec", null, codec);
     
     codec = factory.getCodec(new Path("/tmp/foo.gz"));
-    assertEquals("empty codec gz codec", null, codec);
+    checkCodec("empty factory gz codec", GzipCodec.class, codec);
     codec = factory.getCodecByClassName(GzipCodec.class.getCanonicalName());
-    assertEquals("empty codec gz codec", null, codec);
+    checkCodec("empty factory gz codec", GzipCodec.class, codec);
     
     codec = factory.getCodec(new Path("/tmp/foo.bz2"));
-    assertEquals("empty factory for .bz2", null, codec);
+    checkCodec("empty factory for .bz2", BZip2Codec.class, codec);
     codec = factory.getCodecByClassName(BZip2Codec.class.getCanonicalName());
-    assertEquals("empty factory for bzip2 codec", null, codec);
+    checkCodec("empty factory for bzip2 codec", BZip2Codec.class, codec);
+    
+    codec = factory.getCodec(new Path("/tmp/foo.snappy"));
+    checkCodec("empty factory snappy codec", SnappyCodec.class, codec);
+    codec = factory.getCodecByClassName(SnappyCodec.class.getCanonicalName());
+    checkCodec("empty factory snappy codec", SnappyCodec.class, codec);
+    
+    codec = factory.getCodec(new Path("/tmp/foo.lz4"));
+    checkCodec("empty factory lz4 codec", Lz4Codec.class, codec);
+    codec = factory.getCodecByClassName(Lz4Codec.class.getCanonicalName());
+    checkCodec("empty factory lz4 codec", Lz4Codec.class, codec);
     
     factory = setClasses(new Class[]{BarCodec.class, FooCodec.class, 
                                      FooBarCodec.class});
     codec = factory.getCodec(new Path("/tmp/.foo.bar.gz"));
-    assertEquals("full factory gz codec", null, codec);
+    checkCodec("full factory gz codec", GzipCodec.class, codec);
     codec = factory.getCodecByClassName(GzipCodec.class.getCanonicalName());
-    assertEquals("full codec gz codec", null, codec);
+    checkCodec("full codec gz codec", GzipCodec.class, codec);
      
     codec = factory.getCodec(new Path("/tmp/foo.bz2"));
-    assertEquals("full factory for .bz2", null, codec);
+    checkCodec("full factory for .bz2", BZip2Codec.class, codec);
     codec = factory.getCodecByClassName(BZip2Codec.class.getCanonicalName());
-    assertEquals("full codec bzip2 codec", null, codec);
+    checkCodec("full codec bzip2 codec", BZip2Codec.class, codec);
 
     codec = factory.getCodec(new Path("/tmp/foo.bar"));
     checkCodec("full factory bar codec", BarCodec.class, codec);
@@ -220,5 +237,11 @@ public class TestCodecFactory extends TestCase {
     checkCodec("full factory foo codec", FooCodec.class, codec);
     codec = factory.getCodecByName("FOO");
     checkCodec("full factory foo codec", FooCodec.class, codec);
+    
+    factory = setClasses(new Class[]{NewGzipCodec.class});
+    codec = factory.getCodec(new Path("/tmp/foo.gz"));
+    checkCodec("overridden factory for .gz", NewGzipCodec.class, codec);
+    codec = factory.getCodecByClassName(NewGzipCodec.class.getCanonicalName());
+    checkCodec("overridden factory for gzip codec", NewGzipCodec.class, codec);
   }
 }

+ 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)) {

+ 1 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.net.NetUtils;
 
@@ -99,7 +98,7 @@ public class TestIPC {
     }
 
     @Override
-    public Writable call(RpcKind rpcKind, String protocol, Writable param,
+    public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
         long receiveTime) throws IOException {
       if (sleep) {
         // sleep a bit

+ 1 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 
 /**
@@ -73,7 +72,7 @@ public class TestIPCServerResponder extends TestCase {
     }
 
     @Override
-    public Writable call(RpcKind rpcKind, String protocol, Writable param,
+    public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
         long receiveTime) throws IOException {
       if (sleep) {
         try {

+ 4 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java

@@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.TestProtoBufRpc.PBServerImpl;
 import org.apache.hadoop.ipc.TestProtoBufRpc.TestRpcService;
 import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
@@ -178,9 +177,9 @@ public class TestMultipleProtocolServer {
     // create a server with two handlers
     server = RPC.getServer(Foo0.class,
                               new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
-    server.addProtocol(RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
-    server.addProtocol(RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
-    server.addProtocol(RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
     
     
     // Add Protobuf server
@@ -189,7 +188,7 @@ public class TestMultipleProtocolServer {
         new PBServerImpl();
     BlockingService service = TestProtobufRpcProto
         .newReflectiveBlockingService(pbServerImpl);
-    server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
+    server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
         service);
     server.start();
     addr = NetUtils.getConnectAddress(server);

+ 1 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java

@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
@@ -122,7 +121,7 @@ public class TestProtoBufRpc {
     BlockingService service2 = TestProtobufRpc2Proto
         .newReflectiveBlockingService(server2Impl);
     
-    server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
+    server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
         service2);
     server.start();
   }

+ 9 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java

@@ -31,7 +31,6 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
@@ -134,7 +133,7 @@ public class TestRPCCompatibility {
     TestImpl1 impl = new TestImpl1();
     server = RPC.getServer(TestProtocol1.class,
                             impl, ADDRESS, 0, 2, false, conf, null);
-    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
     addr = NetUtils.getConnectAddress(server);
 
@@ -201,7 +200,7 @@ System.out.println("echo int is NOT supported");
     TestImpl1 impl = new TestImpl1();
     server = RPC.getServer(TestProtocol1.class,
                               impl, ADDRESS, 0, 2, false, conf, null);
-    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
     addr = NetUtils.getConnectAddress(server);
 
@@ -222,7 +221,7 @@ System.out.println("echo int is NOT supported");
     TestImpl2 impl = new TestImpl2();
     server = RPC.getServer(TestProtocol2.class,
                              impl, ADDRESS, 0, 2, false, conf, null);
-    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
     addr = NetUtils.getConnectAddress(server);
 
@@ -316,11 +315,11 @@ System.out.println("echo int is NOT supported");
     TestProtocol2 proxy = RPC.getProxy(TestProtocol2.class,
         TestProtocol2.versionID, addr, conf);
     boolean supported = RpcClientUtil.isMethodSupported(proxy,
-        TestProtocol2.class, RpcKind.RPC_WRITABLE,
+        TestProtocol2.class, RPC.RpcKind.RPC_WRITABLE,
         RPC.getProtocolVersion(TestProtocol2.class), "echo");
     Assert.assertTrue(supported);
     supported = RpcClientUtil.isMethodSupported(proxy,
-        TestProtocol2.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        TestProtocol2.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(TestProtocol2.class), "echo");
     Assert.assertFalse(supported);
   }
@@ -334,7 +333,7 @@ System.out.println("echo int is NOT supported");
     TestImpl1 impl = new TestImpl1();
     server = RPC.getServer(TestProtocol1.class, impl, ADDRESS, 0, 2, false,
         conf, null);
-    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
 
     ProtocolMetaInfoServerSideTranslatorPB xlator = 
@@ -343,13 +342,13 @@ System.out.println("echo int is NOT supported");
     GetProtocolSignatureResponseProto resp = xlator.getProtocolSignature(
         null,
         createGetProtocolSigRequestProto(TestProtocol1.class,
-            RpcKind.RPC_PROTOCOL_BUFFER));
+            RPC.RpcKind.RPC_PROTOCOL_BUFFER));
     //No signatures should be found
     Assert.assertEquals(0, resp.getProtocolSignatureCount());
     resp = xlator.getProtocolSignature(
         null,
         createGetProtocolSigRequestProto(TestProtocol1.class,
-            RpcKind.RPC_WRITABLE));
+            RPC.RpcKind.RPC_WRITABLE));
     Assert.assertEquals(1, resp.getProtocolSignatureCount());
     ProtocolSignatureProto sig = resp.getProtocolSignatureList().get(0);
     Assert.assertEquals(TestProtocol1.versionID, sig.getVersion());
@@ -366,7 +365,7 @@ System.out.println("echo int is NOT supported");
   }
   
   private GetProtocolSignatureRequestProto createGetProtocolSigRequestProto(
-      Class<?> protocol, RpcKind rpcKind) {
+      Class<?> protocol, RPC.RpcKind rpcKind) {
     GetProtocolSignatureRequestProto.Builder builder = 
         GetProtocolSignatureRequestProto.newBuilder();
     builder.setProtocol(protocol.getName());

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

@@ -159,7 +159,7 @@ public class HttpFSFileSystem extends FileSystem {
    * Get operations.
    */
   public enum GetOpValues {
-    OPEN, GETFILESTATUS, LISTSTATUS, GETHOMEDIR, GETCONTENTSUMMARY, GETFILECHECKSUM,
+    OPEN, GETFILESTATUS, LISTSTATUS, GETHOMEDIRECTORY, GETCONTENTSUMMARY, GETFILECHECKSUM,
     GETDELEGATIONTOKEN, GETFILEBLOCKLOCATIONS, INSTRUMENTATION
   }
 
@@ -684,7 +684,7 @@ public class HttpFSFileSystem extends FileSystem {
   @Override
   public Path getHomeDirectory() {
     Map<String, String> params = new HashMap<String, String>();
-    params.put(OP_PARAM, GetOpValues.GETHOMEDIR.toString());
+    params.put(OP_PARAM, GetOpValues.GETHOMEDIRECTORY.toString());
     try {
       HttpURLConnection conn = getConnection(HTTP_GET, params, new Path(getUri().toString(), "/"), false);
       validateResponse(conn, HttpURLConnection.HTTP_OK);

+ 3 - 3
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;
@@ -291,7 +291,7 @@ public class HttpFSServer {
           response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
           break;
         }
-        case GETHOMEDIR: {
+        case GETHOMEDIRECTORY: {
           FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
           JSONObject json = fsExecute(user, doAs.value(), command);
           AUDIT_LOG.info("");

+ 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();
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java

@@ -53,7 +53,7 @@ public class TestCheckUploadContentTypeFilter {
 
   @Test
   public void getOther() throws Exception {
-    test("GET", HttpFSFileSystem.GetOpValues.GETHOMEDIR.toString(), "plain/text", false, false);
+    test("GET", HttpFSFileSystem.GetOpValues.GETHOMEDIRECTORY.toString(), "plain/text", false, false);
   }
 
   @Test

+ 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];

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

@@ -65,6 +65,13 @@ Trunk (unreleased changes)
     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)
+
+    HDFS-3282. Expose getFileLength API. (umamahesh)
+
+    HADOOP-8285 HDFS changes for Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
+
   OPTIMIZATIONS
 
     HDFS-2834. Add a ByteBuffer-based read API to DFSInputStream.
@@ -382,6 +389,17 @@ Release 2.0.0 - UNRELEASED
     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)
+
+    HDFS-3206. Miscellaneous xml cleanups for OEV.
+    (Colin Patrick McCabe via eli)
+
+    HDFS-3169. TestFsck should test multiple -move operations in a row.
+    (Colin Patrick McCabe via eli)
+
   OPTIMIZATIONS
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -527,6 +545,14 @@ Release 2.0.0 - UNRELEASED
     HDFS-3165. HDFS Balancer scripts are refering to wrong path of
     hadoop-daemon.sh (Amith D K via eli)
 
+    HDFS-891. DataNode no longer needs to check for dfs.network.script.
+    (harsh via eli)
+
+    HDFS-3305. GetImageServlet should consider SBN a valid requestor in a
+    secure HA setup. (atm)
+
+    HDFS-3314. HttpFS operation for getHomeDirectory is incorrect. (tucu)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
@@ -860,6 +886,15 @@ 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)
+
+    HDFS-3308. Uses canonical URI to select delegation tokens in HftpFileSystem
+    and WebHdfsFileSystem.  (Daryn Sharp via szetszwo)
+
+    HDFS-3312. In HftpFileSystem, the namenode URI is non-secure but the
+    delegation tokens have to use secure URI.  (Daryn Sharp via szetszwo)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 6 - 34
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -78,7 +78,6 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
@@ -91,6 +90,7 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -1809,41 +1809,13 @@ public class DFSClient implements java.io.Closeable {
   }
 
   /**
-   * The Hdfs implementation of {@link FSDataInputStream}
+   * @deprecated use {@link HdfsDataInputStream} instead.
    */
-  @InterfaceAudience.Private
-  public static class DFSDataInputStream extends FSDataInputStream {
-    public DFSDataInputStream(DFSInputStream in)
-      throws IOException {
-      super(in);
-    }
-      
-    /**
-     * Returns the datanode from which the stream is currently reading.
-     */
-    public DatanodeInfo getCurrentDatanode() {
-      return ((DFSInputStream)in).getCurrentDatanode();
-    }
-      
-    /**
-     * Returns the block containing the target position. 
-     */
-    public ExtendedBlock getCurrentBlock() {
-      return ((DFSInputStream)in).getCurrentBlock();
-    }
+  @Deprecated
+  public static class DFSDataInputStream extends HdfsDataInputStream {
 
-    /**
-     * Return collection of blocks that has already been located.
-     */
-    synchronized List<LocatedBlock> getAllBlocks() throws IOException {
-      return ((DFSInputStream)in).getAllBlocks();
-    }
-    
-    /**
-     * @return The visible length of the file.
-     */
-    public long getVisibleLength() throws IOException {
-      return ((DFSInputStream)in).getFileLength();
+    public DFSDataInputStream(DFSInputStream in) throws IOException {
+      super(in);
     }
   }
   

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

@@ -224,7 +224,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
   /**
    * Return collection of blocks that has already been located.
    */
-  synchronized List<LocatedBlock> getAllBlocks() throws IOException {
+  public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
     return getBlockRange(0, getFileLength());
   }
 

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

@@ -53,7 +53,6 @@ import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1011,7 +1010,7 @@ public class DFSUtil {
   public static void addPBProtocol(Configuration conf, Class<?> protocol,
       BlockingService service, RPC.Server server) throws IOException {
     RPC.setProtocolEngine(conf, protocol, ProtobufRpcEngine.class);
-    server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
+    server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
   }
 
   /**

+ 4 - 45
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -194,8 +195,9 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.recoverLease(getPathName(f));
   }
 
+  @SuppressWarnings("deprecation")
   @Override
-  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+  public HdfsDataInputStream open(Path f, int bufferSize) throws IOException {
     statistics.incrementReadOps(1);
     return new DFSClient.DFSDataInputStream(
           dfs.open(getPathName(f), bufferSize, verifyChecksum));
@@ -499,56 +501,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.
@@ -667,6 +625,7 @@ public class DistributedFileSystem extends FileSystem {
   // We do not see a need for user to report block checksum errors and do not  
   // want to rely on user to report block corruptions.
   @Deprecated
+  @SuppressWarnings("deprecation")
   public boolean reportChecksumFailure(Path f, 
     FSDataInputStream in, long inPos, 
     FSDataInputStream sums, long sumsPos) {

+ 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);
   }
 
   /**

+ 34 - 21
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;
@@ -94,8 +94,8 @@ public class HftpFileSystem extends FileSystem
   protected UserGroupInformation ugi;
   private URI hftpURI;
 
-  protected InetSocketAddress nnAddr;
-  protected InetSocketAddress nnSecureAddr;
+  protected URI nnUri;
+  protected URI nnSecureUri;
 
   public static final String HFTP_TIMEZONE = "UTC";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
@@ -139,11 +139,19 @@ public class HftpFileSystem extends FileSystem
     return NetUtils.createSocketAddrForHost(uri.getHost(), getDefaultSecurePort());
   }
 
+  protected URI getNamenodeUri(URI uri) {
+    return DFSUtil.createUri("http", getNamenodeAddr(uri));
+  }
+
+  protected URI getNamenodeSecureUri(URI uri) {
+    return DFSUtil.createUri("https", getNamenodeSecureAddr(uri));
+  }
+
   @Override
   public String getCanonicalServiceName() {
     // unlike other filesystems, hftp's service is the secure port, not the
     // actual port in the uri
-    return SecurityUtil.buildTokenService(nnSecureAddr).toString();
+    return SecurityUtil.buildTokenService(nnSecureUri).toString();
   }
 
   @Override
@@ -152,8 +160,8 @@ public class HftpFileSystem extends FileSystem
     super.initialize(name, conf);
     setConf(conf);
     this.ugi = UserGroupInformation.getCurrentUser(); 
-    this.nnAddr = getNamenodeAddr(name);
-    this.nnSecureAddr = getNamenodeSecureAddr(name);
+    this.nnUri = getNamenodeUri(name);
+    this.nnSecureUri = getNamenodeSecureUri(name);
     try {
       this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
                              null, null, null);
@@ -168,10 +176,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(ugi);
 
     // if we don't already have a token, go get one over https
     boolean createdToken = false;
@@ -192,14 +197,9 @@ 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(
+      UserGroupInformation ugi) {
+  	return hftpTokenSelector.selectToken(nnSecureUri, ugi.getTokens(), getConf());
   }
   
 
@@ -230,7 +230,7 @@ public class HftpFileSystem extends FileSystem
       ugi.reloginFromKeytab();
       return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
         public Token<?> run() throws IOException {
-          final String nnHttpUrl = DFSUtil.createUri("https", nnSecureAddr).toString();
+          final String nnHttpUrl = nnSecureUri.toString();
           Credentials c;
           try {
             c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
@@ -272,8 +272,8 @@ public class HftpFileSystem extends FileSystem
    * @throws IOException on error constructing the URL
    */
   protected URL getNamenodeURL(String path, String query) throws IOException {
-    final URL url = new URL("http", nnAddr.getHostName(),
-          nnAddr.getPort(), path + '?' + query);
+    final URL url = new URL("http", nnUri.getHost(),
+          nnUri.getPort(), path + '?' + query);
     if (LOG.isTraceEnabled()) {
       LOG.trace("url=" + url);
     }
@@ -699,9 +699,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;
+    }
   }
 }

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

@@ -132,12 +132,17 @@ public class HsftpFileSystem extends HftpFileSystem {
     return getNamenodeAddr(uri);
   }
 
+  @Override
+  protected URI getNamenodeUri(URI uri) {
+    return getNamenodeSecureUri(uri);
+  }
+  
   @Override
   protected HttpURLConnection openConnection(String path, String query)
       throws IOException {
     query = addDelegationTokenParam(query);
-    final URL url = new URL("https", nnAddr.getHostName(), 
-        nnAddr.getPort(), path + '?' + query);
+    final URL url = new URL("https", nnUri.getHost(), 
+        nnUri.getPort(), path + '?' + query);
     HttpsURLConnection conn = (HttpsURLConnection)URLUtils.openConnection(url);
     // bypass hostname verification
     conn.setHostnameVerifier(new DummyHostnameVerifier());

+ 71 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java

@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+
+/**
+ * The Hdfs implementation of {@link FSDataInputStream}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HdfsDataInputStream extends FSDataInputStream {
+  public HdfsDataInputStream(DFSInputStream in) throws IOException {
+    super(in);
+  }
+
+  /**
+   * Get the datanode from which the stream is currently reading.
+   */
+  public DatanodeInfo getCurrentDatanode() {
+    return ((DFSInputStream) in).getCurrentDatanode();
+  }
+
+  /**
+   * Get the block containing the target position.
+   */
+  public ExtendedBlock getCurrentBlock() {
+    return ((DFSInputStream) in).getCurrentBlock();
+  }
+
+  /**
+   * Get the collection of blocks that has already been located.
+   */
+  public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
+    return ((DFSInputStream) in).getAllBlocks();
+  }
+
+  /**
+   * Get the visible length of the file. It will include the length of the last
+   * block even if that is in UnderConstruction state.
+   * 
+   * @return The visible length of the file.
+   */
+  public long getVisibleLength() throws IOException {
+    return ((DFSInputStream) in).getFileLength();
+  }
+}

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -46,7 +46,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -193,7 +192,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy,
-        ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientDatanodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
   }
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -109,7 +109,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 
@@ -812,7 +811,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy,
-        ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
   }
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -69,7 +69,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -308,7 +307,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
   public boolean isMethodSupported(String methodName)
       throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy, DatanodeProtocolPB.class,
-        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(DatanodeProtocolPB.class), methodName);
   }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 
 import com.google.protobuf.RpcController;
@@ -65,7 +64,7 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy,
-        GetUserMappingsProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        GetUserMappingsProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
   }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import com.google.protobuf.RpcController;
@@ -119,7 +118,7 @@ public class InterDatanodeProtocolTranslatorPB implements
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy,
-        InterDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        InterDatanodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(InterDatanodeProtocolPB.class), methodName);
   }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -109,7 +108,7 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy, JournalProtocolPB.class,
-        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(JournalProtocolPB.class), methodName);
   }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -47,7 +47,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -209,7 +208,7 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
   @Override
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy, NamenodeProtocolPB.class,
-        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(NamenodeProtocolPB.class), methodName);
   }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 
 import com.google.protobuf.RpcController;
@@ -64,7 +63,7 @@ public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil.isMethodSupported(rpcProxy,
         RefreshAuthorizationPolicyProtocolPB.class,
-        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
         methodName);
   }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 
 import com.google.protobuf.RpcController;
@@ -76,7 +75,7 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
   public boolean isMethodSupported(String methodName) throws IOException {
     return RpcClientUtil
         .isMethodSupported(rpcProxy, RefreshUserMappingsProtocolPB.class,
-            RpcKind.RPC_PROTOCOL_BUFFER,
+            RPC.RpcKind.RPC_PROTOCOL_BUFFER,
             RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
             methodName);
   }

+ 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() {

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

@@ -1513,11 +1513,6 @@ public class DataNode extends Configured
       printUsage();
       return null;
     }
-    if (conf.get("dfs.network.script") != null) {
-      LOG.error("This configuration for rack identification is not supported" +
-          " anymore. RackID resolution is handled by the NameNode.");
-      System.exit(-1);
-    }
     Collection<URI> dataDirs = getStorageDirs(conf);
     UserGroupInformation.setConfiguration(conf);
     SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,

+ 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");
     }
-      }
-    });
   }
 }

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -1160,7 +1160,9 @@ public abstract class FSEditLogOp {
     @Override
     protected void toXml(ContentHandler contentHandler) throws SAXException {
       XMLUtils.addSaxString(contentHandler, "SRC", src);
-      XMLUtils.addSaxString(contentHandler, "USERNAME", username);
+      if (username != null) {
+        XMLUtils.addSaxString(contentHandler, "USERNAME", username);
+      }
       if (groupname != null) {
         XMLUtils.addSaxString(contentHandler, "GROUPNAME", groupname);
       }
@@ -1168,12 +1170,10 @@ public abstract class FSEditLogOp {
     
     @Override void fromXml(Stanza st) throws InvalidXmlException {
       this.src = st.getValue("SRC");
-      this.username = st.getValue("USERNAME");
-      if (st.hasChildren("GROUPNAME")) {
-        this.groupname = st.getValue("GROUPNAME");
-      } else {
-        this.groupname = null;
-      }
+      this.username = (st.hasChildren("USERNAME")) ? 
+          st.getValue("USERNAME") : null;
+      this.groupname = (st.hasChildren("GROUPNAME")) ? 
+          st.getValue("GROUPNAME") : null;
     }
   }
 
@@ -2346,7 +2346,7 @@ public abstract class FSEditLogOp {
         Long.valueOf(block.getBlockId()).toString());
     XMLUtils.addSaxString(contentHandler, "NUM_BYTES",
         Long.valueOf(block.getNumBytes()).toString());
-    XMLUtils.addSaxString(contentHandler, "GENERATION_STAMP",
+    XMLUtils.addSaxString(contentHandler, "GENSTAMP",
         Long.valueOf(block.getGenerationStamp()).toString());
     contentHandler.endElement("", "", "BLOCK");
   }
@@ -2355,7 +2355,7 @@ public abstract class FSEditLogOp {
       throws InvalidXmlException {
     long blockId = Long.valueOf(st.getValue("BLOCK_ID"));
     long numBytes = Long.valueOf(st.getValue("NUM_BYTES"));
-    long generationStamp = Long.valueOf(st.getValue("GENERATION_STAMP"));
+    long generationStamp = Long.valueOf(st.getValue("GENSTAMP"));
     return new Block(blockId, numBytes, generationStamp);
   }
 

+ 27 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -35,6 +35,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
@@ -44,6 +46,7 @@ import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
@@ -218,26 +221,44 @@ public class GetImageServlet extends HttpServlet {
     return throttler;
   }
   
-  protected boolean isValidRequestor(String remoteUser, Configuration conf)
+  @VisibleForTesting
+  static boolean isValidRequestor(String remoteUser, Configuration conf)
       throws IOException {
     if(remoteUser == null) { // This really shouldn't happen...
       LOG.warn("Received null remoteUser while authorizing access to getImage servlet");
       return false;
     }
+    
+    Set<String> validRequestors = new HashSet<String>();
 
-    String[] validRequestors = {
+    validRequestors.add(
         SecurityUtil.getServerPrincipal(conf
             .get(DFSConfigKeys.DFS_NAMENODE_KRB_HTTPS_USER_NAME_KEY), NameNode
-            .getAddress(conf).getHostName()),
+            .getAddress(conf).getHostName()));
+    validRequestors.add(
         SecurityUtil.getServerPrincipal(conf
             .get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY), NameNode
-            .getAddress(conf).getHostName()),
+            .getAddress(conf).getHostName()));
+    validRequestors.add(
         SecurityUtil.getServerPrincipal(conf
             .get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KRB_HTTPS_USER_NAME_KEY),
-            SecondaryNameNode.getHttpAddress(conf).getHostName()),
+            SecondaryNameNode.getHttpAddress(conf).getHostName()));
+    validRequestors.add(
         SecurityUtil.getServerPrincipal(conf
             .get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_USER_NAME_KEY),
-            SecondaryNameNode.getHttpAddress(conf).getHostName()) };
+            SecondaryNameNode.getHttpAddress(conf).getHostName()));
+
+    if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf))) {
+      Configuration otherNnConf = HAUtil.getConfForOtherNode(conf);
+      validRequestors.add(
+          SecurityUtil.getServerPrincipal(otherNnConf
+              .get(DFSConfigKeys.DFS_NAMENODE_KRB_HTTPS_USER_NAME_KEY),
+              NameNode.getAddress(otherNnConf).getHostName()));
+      validRequestors.add(
+          SecurityUtil.getServerPrincipal(otherNnConf
+              .get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY),
+              NameNode.getAddress(otherNnConf).getHostName()));
+    }
 
     for(String v : validRequestors) {
       if(v != null && v.equals(remoteUser)) {

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

@@ -171,7 +171,8 @@ public class NameNode {
     DFS_NAMENODE_BACKUP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY,
-    DFS_HA_FENCE_METHODS_KEY
+    DFS_HA_FENCE_METHODS_KEY,
+    DFS_NAMENODE_USER_NAME_KEY
   };
   
   public long getProtocolVersion(String protocol, 

+ 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");
+    }
+  }
 }

+ 30 - 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.createSocketAddr(uri.getAuthority(), getDefaultPort());
     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(ugi);
 
     //since we don't already have a token, go get one
     boolean createdToken = false;
@@ -200,6 +196,11 @@ public class WebHdfsFileSystem extends FileSystem
     }
   }
 
+  protected Token<DelegationTokenIdentifier> selectDelegationToken(
+      UserGroupInformation ugi) {
+    return DT_SELECTOR.selectToken(getCanonicalUri(), ugi.getTokens(), getConf());
+  }
+
   @Override
   protected int getDefaultPort() {
     return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
@@ -845,4 +846,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;
+    }
+  }
 }

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

@@ -94,6 +94,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;
@@ -1058,16 +1059,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();

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

@@ -63,7 +63,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.mockito.internal.stubbing.answers.ThrowsException;
@@ -100,7 +99,7 @@ public class TestDFSClientRetries extends TestCase {
     }
 
     @Override
-    public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
+    public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param, long receiveTime)
         throws IOException {
       if (sleep) {
         // sleep a bit

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

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
 import static 
   org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
 
+import java.io.IOException;
 import java.lang.reflect.Field;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
@@ -31,6 +32,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 +68,153 @@ public class TestHftpDelegationToken {
     renewToken.setAccessible(true);
     assertSame("wrong token", token, renewToken.get(fs));
   }
+
+  @Test
+  public void testSelectHftpDelegationToken() throws Exception {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    Configuration conf = new Configuration();
+    conf.setClass("fs.hftp.impl", MyHftpFileSystem.class, FileSystem.class);
+    
+    int httpPort = 80;
+    int httpsPort = 443;
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, httpPort);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, httpsPort);
+    
+    // test with implicit default port 
+    URI fsUri = URI.create("hftp://localhost");
+    MyHftpFileSystem fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    assertEquals(httpPort, fs.getCanonicalUri().getPort());
+    checkTokenSelection(fs, httpsPort, conf); // should still use secure port
+
+    // test with explicit default port
+    fsUri = URI.create("hftp://localhost:"+httpPort);
+    fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    assertEquals(httpPort, fs.getCanonicalUri().getPort());
+    checkTokenSelection(fs, httpsPort, conf); // should still use secure port
+    
+    // test with non-default port
+    fsUri = URI.create("hftp://localhost:"+(httpPort+1));
+    fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
+    checkTokenSelection(fs, httpsPort, conf); // should still use secure port
+    
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 5);
+  }
+
+  @Test
+  public void testSelectHsftpDelegationToken() throws Exception {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    Configuration conf = new Configuration();
+    conf.setClass("fs.hsftp.impl", MyHsftpFileSystem.class, FileSystem.class);
+
+    int httpPort = 80;
+    int httpsPort = 443;
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, httpPort);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, httpsPort);
+
+    // test with implicit default port 
+    URI fsUri = URI.create("hsftp://localhost");
+    MyHsftpFileSystem fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    assertEquals(httpsPort, fs.getCanonicalUri().getPort());
+    checkTokenSelection(fs, httpsPort, conf); 
+
+    // test with explicit default port
+    fsUri = URI.create("hsftp://localhost:"+httpsPort);
+    fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    assertEquals(httpsPort, fs.getCanonicalUri().getPort());
+    checkTokenSelection(fs, httpsPort, conf);
+    
+    // test with non-default port
+    fsUri = URI.create("hsftp://localhost:"+(httpsPort+1));
+    fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    assertEquals(httpsPort+1, fs.getCanonicalUri().getPort());
+    checkTokenSelection(fs, httpsPort+1, conf);
+    
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 5);
+  }
+  
+  private void checkTokenSelection(HftpFileSystem fs,
+                                   int port,
+                                   Configuration conf) throws IOException {
+    UserGroupInformation ugi =
+        UserGroupInformation.createUserForTesting(fs.getUri().getAuthority(), new String[]{});
+
+    // use ip-based tokens
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    // 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);
+
+    // test fallback to hdfs token
+    Token<?> token = fs.selectDelegationToken(ugi);
+    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:"+port));
+    ugi.addToken(hftpToken);
+    token = fs.selectDelegationToken(ugi);
+    assertNotNull(token);
+    assertEquals(hftpToken, token);
+    
+    // switch to using host-based tokens, no token should match
+    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    token = fs.selectDelegationToken(ugi);
+    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(ugi);
+    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:"+port));
+    ugi.addToken(hftpToken);
+    token = fs.selectDelegationToken(ugi);
+    assertNotNull(token);
+    assertEquals(hftpToken, token);
+  }
+  
+  static class MyHftpFileSystem extends HftpFileSystem {
+    @Override
+    public URI getCanonicalUri() {
+      return super.getCanonicalUri();
+    }
+    @Override
+    public int getDefaultPort() {
+      return super.getDefaultPort();
+    }
+    // don't automatically get a token
+    @Override
+    protected void initDelegationToken() throws IOException {}
+  }
+  
+  static class MyHsftpFileSystem extends HsftpFileSystem {
+    @Override
+    public URI getCanonicalUri() {
+      return super.getCanonicalUri();
+    }
+    @Override
+    public int getDefaultPort() {
+      return super.getDefaultPort();
+    }
+    // don't automatically get a token
+    @Override
+    protected void initDelegationToken() throws IOException {}
+  }
 }

+ 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

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

@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.Assert;
@@ -86,7 +85,7 @@ public class TestInterDatanodeProtocol {
     }
 
     @Override
-    public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
+    public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param, long receiveTime)
         throws IOException {
       if (sleep) {
         // sleep a bit

+ 12 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -228,6 +228,7 @@ public class TestFsck extends TestCase {
   }
 
   public void testFsckMoveAndDelete() throws Exception {
+    final int MAX_MOVE_TRIES = 5;
     DFSTestUtil util = new DFSTestUtil("TestFsck", 5, 3, 8*1024);
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
@@ -269,17 +270,19 @@ public class TestFsck extends TestCase {
       } 
       
       // After a fsck -move, the corrupted file should still exist.
-      outStr = runFsck(conf, 1, true, "/", "-move" );
-      assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
-      String[] newFileNames = util.getFileNames(topDir);
-      boolean found = false;
-      for (String f : newFileNames) {
-        if (f.equals(corruptFileName)) {
-          found = true;
-          break;
+      for (int i = 0; i < MAX_MOVE_TRIES; i++) {
+        outStr = runFsck(conf, 1, true, "/", "-move" );
+        assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
+        String[] newFileNames = util.getFileNames(topDir);
+        boolean found = false;
+        for (String f : newFileNames) {
+          if (f.equals(corruptFileName)) {
+            found = true;
+            break;
+          }
         }
+        assertTrue(found);
       }
-      assertTrue(found);
 
       // Fix the filesystem by moving corrupted files to lost+found
       outStr = runFsck(conf, 1, true, "/", "-move", "-delete");

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java

@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.junit.Test;
+
+public class TestGetImageServlet {
+  
+  @Test
+  public void testIsValidRequestorWithHa() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    
+    // Set up generic HA configs.
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "ns1");
+    conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
+        "ns1"), "nn1,nn2");
+    
+    // Set up NN1 HA configs.
+    conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY,
+        "ns1", "nn1"), "host1:1234");
+    conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+        "ns1", "nn1"), "hdfs/_HOST@TEST-REALM.COM");
+    
+    // Set up NN2 HA configs.
+    conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY,
+        "ns1", "nn2"), "host2:1234");
+    conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+        "ns1", "nn2"), "hdfs/_HOST@TEST-REALM.COM");
+    
+    // Initialize this conf object as though we're running on NN1.
+    NameNode.initializeGenericKeys(conf, "ns1", "nn1");
+    
+    // Make sure that NN2 is considered a valid fsimage/edits requestor.
+    assertTrue(GetImageServlet.isValidRequestor("hdfs/host2@TEST-REALM.COM",
+        conf));
+  }
+}

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

@@ -26,6 +26,7 @@ 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;
@@ -44,11 +45,13 @@ 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;
 
@@ -100,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");
@@ -187,23 +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 = SecurityUtil.buildTokenService(addr);
-    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());
+    }    
   }
 
   /**

+ 101 - 1
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,98 @@ public class TestWebHdfsUrl {
   private String generateUrlQueryPrefix(HttpOpParam.Op op, String username) {
     return "op=" + op.toString() + "&user.name=" + username;
   }
-}
+  
+  @Test
+  public void testSelectHdfsDelegationToken() throws Exception {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    Configuration conf = new Configuration();
+    conf.setClass("fs.webhdfs.impl", MyWebHdfsFileSystem.class, FileSystem.class);
+    
+    // test with implicit default port 
+    URI fsUri = URI.create("webhdfs://localhost");
+    MyWebHdfsFileSystem fs = (MyWebHdfsFileSystem) FileSystem.get(fsUri, conf);
+    checkTokenSelection(fs, conf);
+
+    // test with explicit default port
+    fsUri = URI.create("webhdfs://localhost:"+fs.getDefaultPort());
+    fs = (MyWebHdfsFileSystem) FileSystem.get(fsUri, conf);
+    checkTokenSelection(fs, conf);
+    
+    // test with non-default port
+    fsUri = URI.create("webhdfs://localhost:"+(fs.getDefaultPort()-1));
+    fs = (MyWebHdfsFileSystem) FileSystem.get(fsUri, conf);
+    checkTokenSelection(fs, conf);
+
+  }
+  
+  private void checkTokenSelection(MyWebHdfsFileSystem fs,
+                                   Configuration conf) throws IOException {
+    int port = fs.getCanonicalUri().getPort();
+    // can't clear tokens from ugi, so create a new user everytime
+    UserGroupInformation ugi =
+        UserGroupInformation.createUserForTesting(fs.getUri().getAuthority(), new String[]{});
+
+    // use ip-based tokens
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    // 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);
+
+    // test fallback to hdfs token
+    Token<?> token = fs.selectDelegationToken(ugi);
+    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:"+port));
+    ugi.addToken(webHdfsToken);
+    token = fs.selectDelegationToken(ugi);
+    assertNotNull(token);
+    assertEquals(webHdfsToken, token);
+    
+    // switch to using host-based tokens, no token should match
+    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    token = fs.selectDelegationToken(ugi);
+    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(ugi);
+    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:"+port));
+    ugi.addToken(webHdfsToken);
+    token = fs.selectDelegationToken(ugi);
+    assertNotNull(token);
+    assertEquals(webHdfsToken, token);
+  }
+  
+  static class MyWebHdfsFileSystem extends WebHdfsFileSystem {
+    @Override
+    public URI getCanonicalUri() {
+      return super.getCanonicalUri();
+    }
+    @Override
+    public int getDefaultPort() {
+      return super.getDefaultPort();
+    }
+    // don't automatically get a token
+    @Override
+    protected void initDelegationToken() throws IOException {}
+  }
+}

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 39 - 31
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -247,7 +247,7 @@
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1004</GENERATION_STAMP>
+        <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -266,12 +266,12 @@
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1004</GENERATION_STAMP>
+        <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4125931756867080767</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1005</GENERATION_STAMP>
+        <GENSTAMP>1005</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -290,17 +290,17 @@
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1004</GENERATION_STAMP>
+        <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4125931756867080767</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1005</GENERATION_STAMP>
+        <GENSTAMP>1005</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>1562413691487277050</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1006</GENERATION_STAMP>
+        <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -319,17 +319,17 @@
       <BLOCK>
         <BLOCK_ID>-7144805496741076283</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1004</GENERATION_STAMP>
+        <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4125931756867080767</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1005</GENERATION_STAMP>
+        <GENSTAMP>1005</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>1562413691487277050</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1006</GENERATION_STAMP>
+        <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
@@ -379,7 +379,7 @@
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1008</GENERATION_STAMP>
+        <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -398,12 +398,12 @@
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1008</GENERATION_STAMP>
+        <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4219431127125026105</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1009</GENERATION_STAMP>
+        <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -422,17 +422,17 @@
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1008</GENERATION_STAMP>
+        <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4219431127125026105</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1009</GENERATION_STAMP>
+        <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-1765119074945211374</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1010</GENERATION_STAMP>
+        <GENSTAMP>1010</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -451,17 +451,17 @@
       <BLOCK>
         <BLOCK_ID>6084289468290363112</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1008</GENERATION_STAMP>
+        <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-4219431127125026105</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1009</GENERATION_STAMP>
+        <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-1765119074945211374</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1010</GENERATION_STAMP>
+        <GENSTAMP>1010</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
@@ -511,7 +511,7 @@
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1012</GENERATION_STAMP>
+        <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -530,12 +530,12 @@
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1012</GENERATION_STAMP>
+        <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-8051065559769974521</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1013</GENERATION_STAMP>
+        <GENSTAMP>1013</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -554,17 +554,17 @@
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1012</GENERATION_STAMP>
+        <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-8051065559769974521</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1013</GENERATION_STAMP>
+        <GENSTAMP>1013</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>3808670437711973616</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1014</GENERATION_STAMP>
+        <GENSTAMP>1014</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -583,17 +583,17 @@
       <BLOCK>
         <BLOCK_ID>-7448471719302683860</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1012</GENERATION_STAMP>
+        <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>-8051065559769974521</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1013</GENERATION_STAMP>
+        <GENSTAMP>1013</GENSTAMP>
       </BLOCK>
       <BLOCK>
         <BLOCK_ID>3808670437711973616</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
-        <GENERATION_STAMP>1014</GENERATION_STAMP>
+        <GENSTAMP>1014</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
@@ -722,7 +722,7 @@
       <BLOCK>
         <BLOCK_ID>-357061736603024522</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1016</GENERATION_STAMP>
+        <GENSTAMP>1016</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -734,7 +734,7 @@
       <BLOCK>
         <BLOCK_ID>-357061736603024522</BLOCK_ID>
         <NUM_BYTES>0</NUM_BYTES>
-        <GENERATION_STAMP>1016</GENERATION_STAMP>
+        <GENSTAMP>1016</GENSTAMP>
       </BLOCK>
     </DATA>
   </RECORD>
@@ -769,7 +769,7 @@
       <BLOCK>
         <BLOCK_ID>-357061736603024522</BLOCK_ID>
         <NUM_BYTES>11</NUM_BYTES>
-        <GENERATION_STAMP>1017</GENERATION_STAMP>
+        <GENSTAMP>1017</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
@@ -779,9 +779,17 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
+    <OPCODE>OP_SET_OWNER</OPCODE>
     <DATA>
       <TXID>59</TXID>
+      <SRC>/file_create</SRC>
+      <GROUPNAME>newGroup</GROUPNAME>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
+    <DATA>
+      <TXID>60</TXID>
     </DATA>
   </RECORD>
 </EDITS>

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

@@ -50,8 +50,16 @@ Trunk (unreleased changes)
     MAPREDUCE-3935. Annotate Counters.Counter and Counters.Group as @Public.
     (tomwhite)
 
+    HADOOP-8285 MR changes for Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
+
   BUG FIXES
 
+    MAPREDUCE-4100. [Gridmix] Bug fixed in compression emulation feature for 
+                    map only jobs. (amarrk)
+
+    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
@@ -143,6 +151,9 @@ Release 2.0.0 - UNRELEASED
 
     MAPREDUCE-4103. Fix HA docs for changes to shell command fencer args (todd)
 
+    MAPREDUCE-4093. Improve RM WebApp start up when proxy address is not set
+    (Devaraj K vai bobby)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -244,6 +255,11 @@ Release 2.0.0 - UNRELEASED
     MAPREDUCE-4008. ResourceManager throws MetricsException on start up 
     saying QueueMetrics MBean already exists (Devaraj K via tgraves)
 
+    MAPREDUCE-3867. MiniMRYarn/MiniYarn uses fixed ports (tucu)
+
+    MAPREDUCE-4141. clover integration broken, also mapreduce poms are 
+    pulling in clover as a dependency. (phunt via tucu)
+
 Release 0.23.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -251,6 +267,9 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4072. User set java.library.path seems to overwrite default
     creating problems native lib loading (Anupam Seth via bobby)
 
+    MAPREDUCE-3812. Lower default allocation sizes, fix allocation 
+    configurations and document them (Harsh J via bobby)
+
   NEW FEATURES
 
   IMPROVEMENTS
@@ -262,6 +281,20 @@ Release 0.23.3 - UNRELEASED
     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)
+
+    MAPREDUCE-4079. Allow MR AppMaster to limit ephemeral port range.
+    (bobby via tgraves)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -350,6 +383,26 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4160. some mrv1 ant tests fail with timeout - due to 4156 
     (tgraves)
 
+    MAPREDUCE-4074. Client continuously retries to RM When RM goes down 
+    before launching Application Master (xieguiming via tgraves)
+
+    MAPREDUCE-4159. Job is running in Uber mode after setting 
+    "mapreduce.job.ubertask.maxreduces" to zero (Devaraj K via bobby)
+
+    MAPREDUCE-4165. Committing is misspelled as commiting in task logs
+    (John Eagles via bobby)
+
+    MAPREDUCE-4129. Lots of unneeded counters log messages (Ahmed Radwan via
+    bobby)
+
+    MAPREDUCE-3947. yarn.app.mapreduce.am.resource.mb not documented 
+    (Devaraj K via bobby)
+
+    MAPREDUCE-4190. Improve web UI for task attempts userlog link (Tom Graves
+    via bobby)
+
+    MAPREDUCE-4133. MR over viewfs is broken (John George via bobby)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -139,7 +139,8 @@ public class MRClientService extends AbstractService
         rpc.getServer(MRClientProtocol.class, protocolHandler, address,
             conf, secretManager,
             conf.getInt(MRJobConfig.MR_AM_JOB_CLIENT_THREAD_COUNT, 
-                MRJobConfig.DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT));
+                MRJobConfig.DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT),
+                MRJobConfig.MR_AM_JOB_CLIENT_PORT_RANGE);
     
     // Enable service authorization?
     if (conf.getBoolean(

+ 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. 
    */

+ 18 - 6
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;
@@ -821,12 +822,12 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
 
     //FIXME: handling multiple reduces within a single AM does not seem to
     //work.
-    // int sysMaxReduces =
-    //     job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
-    int sysMaxReduces = 1;
+    int sysMaxReduces = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
+    boolean isValidUberMaxReduces = (sysMaxReduces == 0)
+        || (sysMaxReduces == 1);
 
     long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
-        fs.getDefaultBlockSize()); // FIXME: this is wrong; get FS from
+        fs.getDefaultBlockSize(this.remoteJobSubmitDir)); // FIXME: this is wrong; get FS from
                                    // [File?]InputFormat and default block size
                                    // from that
 
@@ -855,7 +856,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     // while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
     // and thus requires sequential execution.
     isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
-        && smallInput && smallMemory && notChainJob;
+        && smallInput && smallMemory && notChainJob && isValidUberMaxReduces;
 
     if (isUber) {
       LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
@@ -888,7 +889,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       if (!smallMemory)
         msg.append(" too much RAM;");
       if (!notChainJob)
-        msg.append(" chainjob");
+        msg.append(" chainjob;");
+      if (!isValidUberMaxReduces)
+        msg.append(" not supported uber max reduces");
       LOG.info(msg.toString());
     }
   }
@@ -1472,4 +1475,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;
+  }
 }

+ 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.

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

@@ -66,7 +66,8 @@ public class TaskPage extends AppView {
             th(".id", "Attempt").
             th(".progress", "Progress").
             th(".state", "State").
-            th(".node", "node").
+            th(".node", "Node").
+            th(".logs", "Logs").
             th(".tsh", "Started").
             th(".tsh", "Finished").
             th(".tsh", "Elapsed").
@@ -83,24 +84,28 @@ public class TaskPage extends AppView {
         long finishTime = ta.getFinishTime();
         long elapsed = ta.getElapsedTime();
         String diag = ta.getNote() == null ? "" : ta.getNote();
-        TD<TR<TBODY<TABLE<Hamlet>>>> nodeTd = tbody.
-          tr().
-            td(".id", taid).
-            td(".progress", progress).
-            td(".state", ta.getState()).td();
+        TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
+        TD<TR<TBODY<TABLE<Hamlet>>>> nodeTd = row.
+          td(".id", taid).
+          td(".progress", progress).
+          td(".state", ta.getState()).td();
         if (nodeHttpAddr == null) {
           nodeTd._("N/A");
         } else {
           nodeTd.
             a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
         }
+        nodeTd._();
         if (containerId != null) {
           String containerIdStr = ta.getAssignedContainerIdStr();
-          nodeTd._(" ").
+          row.td().
             a(".logslink", url("http://", nodeHttpAddr, "node", "containerlogs",
-              containerIdStr, app.getJob().getUserName()), "logs");
+              containerIdStr, app.getJob().getUserName()), "logs")._();
+        } else {
+          row.td()._("N/A")._();
         }
-        nodeTd._().
+
+        row.
           td(".ts", Times.format(startTime)).
           td(".ts", Times.format(finishTime)).
           td(".dt", StringUtils.formatTime(elapsed)).

+ 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()));
     }

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

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobACLsManager;
 import org.apache.hadoop.mapred.ShuffleHandler;
@@ -442,7 +444,7 @@ public class MockJobs extends MockApps {
     final Path configFile = confFile;
 
     Map<JobACL, AccessControlList> tmpJobACLs = new HashMap<JobACL, AccessControlList>();
-    Configuration conf = new Configuration();
+    final Configuration conf = new Configuration();
     conf.set(JobACL.VIEW_JOB.getAclName(), "testuser");
     conf.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
 
@@ -564,6 +566,14 @@ public class MockJobs extends MockApps {
         amInfoList.add(createAMInfo(2));
         return amInfoList;
       }
+
+      @Override
+      public Configuration loadConfFile() throws IOException {
+        FileContext fc = FileContext.getFileContext(configFile.toUri(), conf);
+        Configuration jobConf = new Configuration(false);
+        jobConf.addResource(fc.open(configFile));
+        return jobConf;
+      }
     };
   }
 

+ 2 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
@@ -341,7 +342,7 @@ public class TestRMContainerAllocator {
     }
     @Override
     protected ResourceScheduler createScheduler() {
-      return new MyFifoScheduler(getRMContext());
+      return new MyFifoScheduler();
     }
   }
 
@@ -997,18 +998,6 @@ public class TestRMContainerAllocator {
   }
   
   private static class MyFifoScheduler extends FifoScheduler {
-
-    public MyFifoScheduler(RMContext rmContext) {
-      super();
-      try {
-        reinitialize(new Configuration(), new ContainerTokenSecretManager(),
-            rmContext);
-      } catch (IOException ie) {
-        LOG.info("add application failed with ", ie);
-        assert (false);
-      }
-    }
-
     // override this to copy the objects otherwise FifoScheduler updates the
     // numContainers in same objects as kept by RMContainerAllocator
     @Override

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java

@@ -489,6 +489,11 @@ public class TestRuntimeEstimators {
     public List<AMInfo> getAMInfos() {
       throw new UnsupportedOperationException("Not supported yet.");
     }
+    
+    @Override
+    public Configuration loadConfFile() {
+      throw new UnsupportedOperationException();
+    }
   }
 
   /*

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

@@ -37,14 +37,20 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.InitTransition;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.JobNoTasksCompletedTransition;
+import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -233,4 +239,69 @@ public class TestJobImpl {
     Assert.assertTrue(job5.checkAccess(ugi1, null));
     Assert.assertTrue(job5.checkAccess(ugi2, null));
   }
+  @Test
+  public void testUberDecision() throws Exception {
+
+    // with default values, no of maps is 2
+    Configuration conf = new Configuration();
+    boolean isUber = testUberDecision(conf);
+    Assert.assertFalse(isUber);
+
+    // enable uber mode, no of maps is 2
+    conf = new Configuration();
+    conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
+    isUber = testUberDecision(conf);
+    Assert.assertTrue(isUber);
+
+    // enable uber mode, no of maps is 2, no of reduces is 1 and uber task max
+    // reduces is 0
+    conf = new Configuration();
+    conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
+    conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 0);
+    conf.setInt(MRJobConfig.NUM_REDUCES, 1);
+    isUber = testUberDecision(conf);
+    Assert.assertFalse(isUber);
+
+    // enable uber mode, no of maps is 2, no of reduces is 1 and uber task max
+    // reduces is 1
+    conf = new Configuration();
+    conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
+    conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
+    conf.setInt(MRJobConfig.NUM_REDUCES, 1);
+    isUber = testUberDecision(conf);
+    Assert.assertTrue(isUber);
+
+    // enable uber mode, no of maps is 2 and uber task max maps is 0
+    conf = new Configuration();
+    conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
+    conf.setInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 1);
+    isUber = testUberDecision(conf);
+    Assert.assertFalse(isUber);
+  }
+
+  private boolean testUberDecision(Configuration conf) {
+    JobID jobID = JobID.forName("job_1234567890000_0001");
+    JobId jobId = TypeConverter.toYarn(jobID);
+    MRAppMetrics mrAppMetrics = MRAppMetrics.create();
+    JobImpl job = new JobImpl(jobId, Records
+        .newRecord(ApplicationAttemptId.class), conf, mock(EventHandler.class),
+        null, mock(JobTokenSecretManager.class), null, null, null,
+        mrAppMetrics, mock(OutputCommitter.class), true, null, 0, null, null);
+    InitTransition initTransition = getInitTransition();
+    JobEvent mockJobEvent = mock(JobEvent.class);
+    initTransition.transition(job, mockJobEvent);
+    boolean isUber = job.isUber();
+    return isUber;
+  }
+
+  private InitTransition getInitTransition() {
+    InitTransition initTransition = new InitTransition() {
+      @Override
+      protected TaskSplitMetaInfo[] createSplits(JobImpl job, JobId jobId) {
+        return new TaskSplitMetaInfo[] { new TaskSplitMetaInfo(),
+            new TaskSplitMetaInfo() };
+      }
+    };
+    return initTransition;
+  }
 }

+ 26 - 30
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java

@@ -187,9 +187,6 @@ public class JobConf extends Configuration {
    * /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of:
    *          -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc
    * 
-   * The configuration variable {@link #MAPRED_TASK_ULIMIT} can be used to 
-   * control the maximum virtual memory of the child processes.
-   * 
    * The configuration variable {@link #MAPRED_TASK_ENV} can be used to pass 
    * other environment variables to the child processes.
    * 
@@ -210,9 +207,6 @@ public class JobConf extends Configuration {
    * /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of:
    *          -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc
    * 
-   * The configuration variable {@link #MAPRED_MAP_TASK_ULIMIT} can be used to 
-   * control the maximum virtual memory of the map processes.
-   * 
    * The configuration variable {@link #MAPRED_MAP_TASK_ENV} can be used to pass 
    * other environment variables to the map processes.
    */
@@ -230,9 +224,6 @@ public class JobConf extends Configuration {
    * /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of:
    *          -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc
    * 
-   * The configuration variable {@link #MAPRED_REDUCE_TASK_ULIMIT} can be used  
-   * to control the maximum virtual memory of the reduce processes.
-   * 
    * The configuration variable {@link #MAPRED_REDUCE_TASK_ENV} can be used to 
    * pass process environment variables to the reduce processes.
    */
@@ -242,36 +233,32 @@ public class JobConf extends Configuration {
   public static final String DEFAULT_MAPRED_TASK_JAVA_OPTS = "-Xmx200m";
   
   /**
-   * Configuration key to set the maximum virutal memory available to the child
-   * map and reduce tasks (in kilo-bytes).
-   * 
-   * Note: This must be greater than or equal to the -Xmx passed to the JavaVM
-   *       via {@link #MAPRED_TASK_JAVA_OPTS}, else the VM might not start.
-   * 
-   * @deprecated Use {@link #MAPRED_MAP_TASK_ULIMIT} or 
-   *                 {@link #MAPRED_REDUCE_TASK_ULIMIT}
+   * @deprecated
+   * Configuration key to set the maximum virtual memory available to the child
+   * map and reduce tasks (in kilo-bytes). This has been deprecated and will no
+   * longer have any effect.
    */
   @Deprecated
   public static final String MAPRED_TASK_ULIMIT = "mapred.child.ulimit";
 
   /**
-   * Configuration key to set the maximum virutal memory available to the
-   * map tasks (in kilo-bytes).
-   * 
-   * Note: This must be greater than or equal to the -Xmx passed to the JavaVM
-   *       via {@link #MAPRED_MAP_TASK_JAVA_OPTS}, else the VM might not start.
+   * @deprecated
+   * Configuration key to set the maximum virtual memory available to the
+   * map tasks (in kilo-bytes). This has been deprecated and will no
+   * longer have any effect.
    */
-  public static final String MAPRED_MAP_TASK_ULIMIT = JobContext.MAP_ULIMIT;
+  @Deprecated
+  public static final String MAPRED_MAP_TASK_ULIMIT = "mapreduce.map.ulimit";
   
   /**
-   * Configuration key to set the maximum virutal memory available to the
-   * reduce tasks (in kilo-bytes).
-   * 
-   * Note: This must be greater than or equal to the -Xmx passed to the JavaVM
-   *       via {@link #MAPRED_REDUCE_TASK_JAVA_OPTS}, else the VM might not start.
+   * @deprecated
+   * Configuration key to set the maximum virtual memory available to the
+   * reduce tasks (in kilo-bytes). This has been deprecated and will no
+   * longer have any effect.
    */
-  public static final String MAPRED_REDUCE_TASK_ULIMIT = 
-    JobContext.REDUCE_ULIMIT;
+  @Deprecated
+  public static final String MAPRED_REDUCE_TASK_ULIMIT =
+    "mapreduce.reduce.ulimit";
 
 
   /**
@@ -1966,6 +1953,15 @@ public class JobConf extends Configuration {
                 + " Instead use " + JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY
                 + " and " + JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY);
     }
+    if(get(JobConf.MAPRED_TASK_ULIMIT) != null ) {
+      LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_TASK_ULIMIT));
+    }
+    if(get(JobConf.MAPRED_MAP_TASK_ULIMIT) != null ) {
+      LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_MAP_TASK_ULIMIT));
+    }
+    if(get(JobConf.MAPRED_REDUCE_TASK_ULIMIT) != null ) {
+      LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_REDUCE_TASK_ULIMIT));
+    }
   }
   
 

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java

@@ -55,10 +55,10 @@ public class Master {
       return NetUtils.createSocketAddr(masterAddress, 8012, MRConfig.MASTER_ADDRESS);
     } 
     else {
-      masterAddress = conf.get(YarnConfiguration.RM_ADDRESS,
-          YarnConfiguration.DEFAULT_RM_ADDRESS);
-      return NetUtils.createSocketAddr(masterAddress, YarnConfiguration.DEFAULT_RM_PORT,
-        YarnConfiguration.RM_ADDRESS);
+      return conf.getSocketAddr(
+          YarnConfiguration.RM_ADDRESS,
+          YarnConfiguration.DEFAULT_RM_ADDRESS,
+          YarnConfiguration.DEFAULT_RM_PORT);
     }
   }
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java

@@ -964,7 +964,7 @@ abstract public class Task implements Writable, Configurable {
                    TaskReporter reporter
                    ) throws IOException, InterruptedException {
     LOG.info("Task:" + taskId + " is done."
-             + " And is in the process of commiting");
+             + " And is in the process of committing");
     updateCounters();
 
     boolean commitRequired = isCommitRequired();

+ 16 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -183,8 +183,6 @@ public interface MRJobConfig {
 
   public static final String MAP_JAVA_OPTS = "mapreduce.map.java.opts";
 
-  public static final String MAP_ULIMIT = "mapreduce.map.ulimit";
-
   public static final String MAP_MAX_ATTEMPTS = "mapreduce.map.maxattempts";
 
   public static final String MAP_DEBUG_SCRIPT = "mapreduce.map.debug.script";
@@ -243,8 +241,6 @@ public interface MRJobConfig {
 
   public static final String REDUCE_JAVA_OPTS = "mapreduce.reduce.java.opts";
 
-  public static final String REDUCE_ULIMIT = "mapreduce.reduce.ulimit";
-  
   public static final String MAPREDUCE_JOB_DIR = "mapreduce.job.dir";
 
   public static final String REDUCE_MAX_ATTEMPTS = "mapreduce.reduce.maxattempts";
@@ -329,6 +325,13 @@ public interface MRJobConfig {
     MR_PREFIX + "client-am.ipc.max-retries";
   public static final int DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 3;
   
+  /**
+   * The number of client retries to the RM/HS/AM before throwing exception.
+   */
+  public static final String MR_CLIENT_MAX_RETRIES = 
+    MR_PREFIX + "client.max-retries";
+  public static final int DEFAULT_MR_CLIENT_MAX_RETRIES = 3;
+  
   /** The staging directory for map reduce.*/
   public static final String MR_AM_STAGING_DIR = 
     MR_AM_PREFIX+"staging-dir";
@@ -338,12 +341,12 @@ public interface MRJobConfig {
   /** The amount of memory the MR app master needs.*/
   public static final String MR_AM_VMEM_MB =
     MR_AM_PREFIX+"resource.mb";
-  public static final int DEFAULT_MR_AM_VMEM_MB = 2048;
+  public static final int DEFAULT_MR_AM_VMEM_MB = 1536;
 
   /** Command line arguments passed to the MR app master.*/
   public static final String MR_AM_COMMAND_OPTS =
     MR_AM_PREFIX+"command-opts";
-  public static final String DEFAULT_MR_AM_COMMAND_OPTS = "-Xmx1536m";
+  public static final String DEFAULT_MR_AM_COMMAND_OPTS = "-Xmx1024m";
 
   /** Root Logging level passed to the MR app master.*/
   public static final String MR_AM_LOG_LEVEL = 
@@ -370,6 +373,13 @@ public interface MRJobConfig {
     MR_AM_PREFIX + "job.client.thread-count";
   public static final int DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT = 1;
 
+  /** 
+   * Range of ports that the MapReduce AM can use when binding. Leave blank
+   * if you want all possible ports.
+   */
+  public static final String MR_AM_JOB_CLIENT_PORT_RANGE = 
+    MR_AM_PREFIX + "job.client.port-range";
+  
   /** Enable blacklisting of nodes in the job.*/
   public static final String MR_AM_JOB_NODE_BLACKLISTING_ENABLE = 
     MR_AM_PREFIX  + "job.node-blacklisting.enable";

+ 13 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/AbstractCounters.java

@@ -201,7 +201,15 @@ public abstract class AbstractCounters<C extends Counter,
    * @return the group
    */
   public synchronized G getGroup(String groupName) {
-    String newGroupName = filterGroupName(groupName);
+
+    // filterGroupName
+    boolean groupNameInLegacyMap = true;
+    String newGroupName = legacyMap.get(groupName);
+    if (newGroupName == null) {
+      groupNameInLegacyMap = false;
+      newGroupName = Limits.filterGroupName(groupName);
+    }
+
     boolean isFGroup = isFrameworkGroup(newGroupName);
     G group = isFGroup ? fgroups.get(newGroupName) : groups.get(newGroupName);
     if (group == null) {
@@ -212,19 +220,14 @@ public abstract class AbstractCounters<C extends Counter,
         limits.checkGroups(groups.size() + 1);
         groups.put(newGroupName, group);
       }
+      if (groupNameInLegacyMap) {
+        LOG.warn("Group " + groupName + " is deprecated. Use " + newGroupName
+            + " instead");
+      }
     }
     return group;
   }
 
-  private String filterGroupName(String oldName) {
-    String newName = legacyMap.get(oldName);
-    if (newName == null) {
-      return Limits.filterGroupName(oldName);
-    }
-    LOG.warn("Group "+ oldName +" is deprecated. Use "+ newName +" instead");
-    return newName;
-  }
-
   /**
    * Returns the total number of counters, by summing the number of counters
    * in each group.

Alguns arquivos não foram mostrados porque muitos arquivos mudaram nesse diff