浏览代码

Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1568497 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父节点
当前提交
d8fd23fc46
共有 26 个文件被更改,包括 535 次插入130 次删除
  1. 5 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 6 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java
  3. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
  5. 2 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  6. 40 28
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  7. 27 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  8. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  9. 25 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  11. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
  12. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  13. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReader.java
  14. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReader.java
  15. 88 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/lib/TestCombineFileRecordReader.java
  16. 96 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileRecordReader.java
  17. 8 0
      hadoop-yarn-project/CHANGES.txt
  18. 9 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
  19. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  20. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  21. 0 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  22. 1 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java
  23. 1 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  24. 8 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  25. 120 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
  26. 43 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

+ 5 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -315,6 +315,8 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10333. Fix grammatical error in overview.html document.
     HADOOP-10333. Fix grammatical error in overview.html document.
     (René Nyffenegger via suresh)
     (René Nyffenegger via suresh)
 
 
+    HADOOP-10343. Change info to debug log in LossyRetryInvocationHandler. (arpit)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -337,6 +339,9 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10338. Cannot get the FileStatus of the root inode from the new
     HADOOP-10338. Cannot get the FileStatus of the root inode from the new
     Globber (cmccabe)
     Globber (cmccabe)
 
 
+    HADOOP-10249. LdapGroupsMapping should trim ldap password read from file.
+    (Dilli Armugam via suresh)
+
 Release 2.3.1 - UNRELEASED
 Release 2.3.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 6 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/LossyRetryInvocationHandler.java

@@ -51,11 +51,15 @@ public class LossyRetryInvocationHandler<T> extends RetryInvocationHandler<T> {
     int retryCount = RetryCount.get();
     int retryCount = RetryCount.get();
     if (retryCount < this.numToDrop) {
     if (retryCount < this.numToDrop) {
       RetryCount.set(++retryCount);
       RetryCount.set(++retryCount);
-      LOG.info("Drop the response. Current retryCount == " + retryCount);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Drop the response. Current retryCount == " + retryCount);
+      }
       throw new RetriableException("Fake Exception");
       throw new RetriableException("Fake Exception");
     } else {
     } else {
-      LOG.info("retryCount == " + retryCount
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("retryCount == " + retryCount
           + ". It's time to normally process the response");
           + ". It's time to normally process the response");
+      }
       return result;
       return result;
     }
     }
   }
   }

+ 15 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java

@@ -27,6 +27,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -240,4 +241,18 @@ public class Groups {
     }
     }
     return GROUPS;
     return GROUPS;
   }
   }
+
+  /**
+   * Create new groups used to map user-to-groups with loaded configuration.
+   * @param conf
+   * @return the groups being used to map user-to-groups.
+   */
+  @Private
+  public static synchronized Groups
+      getUserToGroupsMappingServiceWithLoadedConfiguration(
+          Configuration conf) {
+
+    GROUPS = new Groups(conf);
+    return GROUPS;
+  }
 }
 }

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

@@ -356,7 +356,7 @@ public class LdapGroupsMapping
         c = reader.read();
         c = reader.read();
       }
       }
       reader.close();
       reader.close();
-      return password.toString();
+      return password.toString().trim();
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       throw new RuntimeException("Could not read password file: " + pwFile, ioe);
       throw new RuntimeException("Could not read password file: " + pwFile, ioe);
     }
     }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -1829,7 +1829,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       } catch (IOException e1) {
       } catch (IOException e1) {
         LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
         LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
       }
       }
-      WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
+      WccData fileWcc = new WccData(preOpAttr == null ? null
+          : Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
       return new COMMIT3Response(Nfs3Status.NFS3ERR_IO, fileWcc,
       return new COMMIT3Response(Nfs3Status.NFS3ERR_IO, fileWcc,
           Nfs3Constant.WRITE_COMMIT_VERF);
           Nfs3Constant.WRITE_COMMIT_VERF);
     }
     }

+ 40 - 28
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -120,31 +120,6 @@ Trunk (Unreleased)
     HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
     HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
     Yamashita via brandonli)
     Yamashita via brandonli)
 
 
-    HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
-    (szetszwo)
-
-    HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
-    and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
-    (jing9 via szetszwo)
-
-    HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
-    with DirectoryWithQuotaFeature.  (szetszwo)
-
-    HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
-
-    HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
-    FileWithSnapshotFeature.  (jing9 via szetszwo)
-
-    HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
-    via jing9)
-
-    HDFS-5632. Flatten INodeDirectory hierarchy: Replace
-    INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
-    (jing9 via szetszwo)
-
-    HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
-    FileDiff/DirectoryDiff. (jing9)
-
     HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be 
     HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be 
     closed before method returns. (Ted Yu via junping_du)
     closed before method returns. (Ted Yu via junping_du)
 
 
@@ -275,8 +250,6 @@ Trunk (Unreleased)
     HDFS-5719. FSImage#doRollback() should close prevState before return
     HDFS-5719. FSImage#doRollback() should close prevState before return
     (Ted Yu via brandonli)
     (Ted Yu via brandonli)
 
 
-    HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
-
     HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager
     HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager
     (Haohui Mai via brandonli)
     (Haohui Mai via brandonli)
 
 
@@ -376,6 +349,33 @@ Release 2.4.0 - UNRELEASED
     HDFS-5940. Minor cleanups to ShortCircuitReplica, FsDatasetCache, and
     HDFS-5940. Minor cleanups to ShortCircuitReplica, FsDatasetCache, and
     DomainSocketWatcher (cmccabe)
     DomainSocketWatcher (cmccabe)
 
 
+    HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
+    (szetszwo)
+
+    HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
+    and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
+    (jing9 via szetszwo)
+
+    HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
+    with DirectoryWithQuotaFeature.  (szetszwo)
+
+    HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
+
+    HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
+    FileWithSnapshotFeature.  (jing9 via szetszwo)
+
+    HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
+    via jing9)
+
+    HDFS-5632. Flatten INodeDirectory hierarchy: Replace
+    INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
+    (jing9 via szetszwo)
+
+    HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
+    FileDiff/DirectoryDiff. (jing9)
+
+    HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@@ -438,7 +438,19 @@ Release 2.4.0 - UNRELEASED
     HDFS-5941. add dfs.namenode.secondary.https-address and
     HDFS-5941. add dfs.namenode.secondary.https-address and
     dfs.namenode.secondary.https-address in hdfs-default.xml.
     dfs.namenode.secondary.https-address in hdfs-default.xml.
     (Haohui Mai via cnauroth)
     (Haohui Mai via cnauroth)
- 
+
+    HDFS-5913. Nfs3Utils#getWccAttr() should check attr parameter against null
+    (brandonli)
+
+    HDFS-5934. New Namenode UI back button doesn't work as expected
+    (Travis Thompson via brandonli)
+
+    HDFS-5901. NameNode new UI doesn't support IE8 and IE9 on windows 7
+    (Vinayakumar B via brandonli)
+
+    HDFS-5943. 'dfs.namenode.https-address' property is not loaded from
+    configuration in federation setup. (suresh)
+
 Release 2.3.1 - UNRELEASED
 Release 2.3.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 27 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -17,39 +17,22 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import javax.management.ObjectName;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceStatus;
 import org.apache.hadoop.ha.HAServiceStatus;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.ha.ServiceFailedException;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Trash;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.apache.hadoop.util.ExitUtil.terminate;
-import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
-
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -58,20 +41,11 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
-import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
-import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
-import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
-import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
+import org.apache.hadoop.hdfs.server.namenode.ha.*;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressMetrics;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressMetrics;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -89,10 +63,23 @@ import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.util.ExitUtil.terminate;
+import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
 
 
 /**********************************************************
 /**********************************************************
  * NameNode serves as both directory namespace manager and
  * NameNode serves as both directory namespace manager and
@@ -183,8 +170,10 @@ public class NameNode implements NameNodeStatusMXBean {
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
     DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_HTTP_ADDRESS_KEY,
+    DFS_NAMENODE_HTTPS_ADDRESS_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+    DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY,
     DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
     DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_BACKUP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -1,3 +1,5 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+    "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <!--
 <!--
    Licensed to the Apache Software Foundation (ASF) under one or more
    Licensed to the Apache Software Foundation (ASF) under one or more
    contributor license agreements.  See the NOTICE file distributed with
    contributor license agreements.  See the NOTICE file distributed with
@@ -14,8 +16,6 @@
    See the License for the specific language governing permissions and
    See the License for the specific language governing permissions and
    limitations under the License.
    limitations under the License.
 -->
 -->
-<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
-    "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <head>
 <head>
 <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />
 <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />

+ 25 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -28,7 +28,7 @@
       {"name": "nn",      "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
       {"name": "nn",      "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
       {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
       {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
       {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
       {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
-      {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"},
+      {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"}
     ];
     ];
 
 
     var HELPERS = {
     var HELPERS = {
@@ -166,14 +166,29 @@
 
 
   $('#ui-tabs a[href="#tab-snapshot"]').click(load_snapshot_info);
   $('#ui-tabs a[href="#tab-snapshot"]').click(load_snapshot_info);
 
 
-  var hash = window.location.hash;
-  if (hash === "#tab-datanode") {
-    load_datanode_info();
-  } else if (hash === "#tab-snapshot") {
-    load_snapshot_info();
-  } else if (hash === "#tab-startup-progress") {
-    load_startup_progress();
-  } else {
-    load_overview();
+  function load_page() {
+    var hash = window.location.hash;
+    switch(hash) {
+      case "#tab-datanode":
+        load_datanode_info();
+        break;
+      case "#tab-snapshot":
+        load_snapshot_info();
+        break;
+      case "#tab-startup-progress":
+        load_startup_progress();
+        break;
+      case "#tab-overview":
+        load_overview();
+        break;
+      default:
+        window.location.hash = "tab-overview";
+        break;
+    }
   }
   }
+  load_page();
+
+  $(window).bind('hashchange', function () {
+    load_page();
+  });
 })();
 })();

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

@@ -1,3 +1,5 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+    "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <!--
 <!--
     Licensed to the Apache Software Foundation (ASF) under one or more
     Licensed to the Apache Software Foundation (ASF) under one or more
     contributor license agreements.  See the NOTICE file distributed with
     contributor license agreements.  See the NOTICE file distributed with
@@ -14,8 +16,6 @@
     See the License for the specific language governing permissions and
     See the License for the specific language governing permissions and
     limitations under the License.
     limitations under the License.
   -->
   -->
-<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
-	  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <html xmlns="http://www.w3.org/1999/xhtml">
   <head>
   <head>
     <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />
     <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -52,6 +52,18 @@
     $('#alert-panel').show();
     $('#alert-panel').show();
   }
   }
 
 
+  $(window).bind('hashchange', function () {
+    $('#alert-panel').hide();
+
+    var dir = window.location.hash.slice(1);
+    if(dir == "") {
+      dir = "/";
+    }
+    if(current_directory != dir) {
+      browse_directory(dir);
+    }
+  });
+
   function network_error_handler(url) {
   function network_error_handler(url) {
     return function (jqxhr, text, err) {
     return function (jqxhr, text, err) {
       var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
       var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
@@ -145,6 +157,7 @@
 
 
       current_directory = dir;
       current_directory = dir;
       $('#directory').val(dir);
       $('#directory').val(dir);
+      window.location.hash = dir;
       dust.render('explorer', base.push(d), function(err, out) {
       dust.render('explorer', base.push(d), function(err, out) {
         $('#panel').html(out);
         $('#panel').html(out);
 
 
@@ -169,7 +182,12 @@
 
 
     var b = function() { browse_directory($('#directory').val()); };
     var b = function() { browse_directory($('#directory').val()); };
     $('#btn-nav-directory').click(b);
     $('#btn-nav-directory').click(b);
-    browse_directory('/');
+    var dir = window.location.hash.slice(1);
+    if(dir == "") {
+      window.location.hash = "/";
+    } else {
+      browse_directory(dir);
+    }
   }
   }
 
 
   init();
   init();

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

@@ -162,6 +162,9 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5746. Job diagnostics can implicate wrong task for a failed job.
     MAPREDUCE-5746. Job diagnostics can implicate wrong task for a failed job.
     (Jason Lowe via kasha)
     (Jason Lowe via kasha)
 
 
+    MAPREDUCE-5670. CombineFileRecordReader should report progress when moving
+    to the next file (Chen He via jlowe)
+
 Release 2.3.1 - UNRELEASED
 Release 2.3.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReader.java

@@ -140,6 +140,8 @@ public class CombineFileRecordReader<K, V> implements RecordReader<K, V> {
       return false;
       return false;
     }
     }
 
 
+    reporter.progress();
+
     // get a record reader for the idx-th chunk
     // get a record reader for the idx-th chunk
     try {
     try {
       curReader =  rrConstructor.newInstance(new Object [] 
       curReader =  rrConstructor.newInstance(new Object [] 

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReader.java

@@ -54,7 +54,7 @@ public class CombineFileRecordReader<K, V> extends RecordReader<K, V> {
   protected int idx;
   protected int idx;
   protected long progress;
   protected long progress;
   protected RecordReader<K, V> curReader;
   protected RecordReader<K, V> curReader;
-  
+
   public void initialize(InputSplit split,
   public void initialize(InputSplit split,
       TaskAttemptContext context) throws IOException, InterruptedException {
       TaskAttemptContext context) throws IOException, InterruptedException {
     this.split = (CombineFileSplit)split;
     this.split = (CombineFileSplit)split;
@@ -144,6 +144,8 @@ public class CombineFileRecordReader<K, V> extends RecordReader<K, V> {
       return false;
       return false;
     }
     }
 
 
+    context.progress();
+
     // get a record reader for the idx-th chunk
     // get a record reader for the idx-th chunk
     try {
     try {
       Configuration conf = context.getConfiguration();
       Configuration conf = context.getConfiguration();

+ 88 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/lib/TestCombineFileRecordReader.java

@@ -0,0 +1,88 @@
+/**
+ * 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.mapred.lib;
+
+import java.io.File;
+import java.io.FileWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.fs.FileUtil;
+
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.junit.Assert;
+
+import java.io.IOException;
+
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class TestCombineFileRecordReader {
+
+  private static Path outDir = new Path(System.getProperty("test.build.data",
+            "/tmp"), TestCombineFileRecordReader.class.getName());
+
+  private static class TextRecordReaderWrapper
+    extends org.apache.hadoop.mapred.lib.CombineFileRecordReaderWrapper<LongWritable,Text> {
+    // this constructor signature is required by CombineFileRecordReader
+    public TextRecordReaderWrapper(CombineFileSplit split, Configuration conf,
+      Reporter reporter, Integer idx) throws IOException {
+      super(new TextInputFormat(), split, conf, reporter, idx);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testInitNextRecordReader() throws IOException{
+    JobConf conf = new JobConf();
+    Path[] paths = new Path[3];
+    long[] fileLength = new long[3];
+    File[] files = new File[3];
+    LongWritable key = new LongWritable(1);
+    Text value = new Text();
+    try {
+      for(int i=0;i<3;i++){
+        fileLength[i] = i;
+        File dir = new File(outDir.toString());
+        dir.mkdir();
+        files[i] = new File(dir,"testfile"+i);
+        FileWriter fileWriter = new FileWriter(files[i]);
+        fileWriter.close();
+        paths[i] = new Path(outDir+"/testfile"+i);
+      }
+      CombineFileSplit combineFileSplit = new CombineFileSplit(conf, paths, fileLength);
+      Reporter reporter = Mockito.mock(Reporter.class);
+      CombineFileRecordReader cfrr = new CombineFileRecordReader(conf, combineFileSplit,
+        reporter,  TextRecordReaderWrapper.class);
+      verify(reporter).progress();
+      Assert.assertFalse(cfrr.next(key,value));
+      verify(reporter, times(3)).progress();
+    } finally {
+      FileUtil.fullyDelete(new File(outDir.toString()));
+    }
+
+  }
+}
+

+ 96 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileRecordReader.java

@@ -0,0 +1,96 @@
+/**
+ * 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.mapreduce.lib.input;
+
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.File;
+
+import junit.framework.Assert;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapred.Task.TaskReporter;
+
+import org.mockito.Mockito;
+
+import org.junit.Test;
+
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class TestCombineFileRecordReader {
+
+  private static Path outDir = new Path(System.getProperty("test.build.data",
+            "/tmp"), TestCombineFileRecordReader.class.getName());
+  private static class TextRecordReaderWrapper
+    extends CombineFileRecordReaderWrapper<LongWritable,Text> {
+    // this constructor signature is required by CombineFileRecordReader
+    public TextRecordReaderWrapper(org.apache.hadoop.mapreduce.lib.input.CombineFileSplit split,
+      TaskAttemptContext context, Integer idx)
+        throws IOException, InterruptedException {
+      super(new TextInputFormat(), split, context, idx);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testProgressIsReportedIfInputASeriesOfEmptyFiles() throws IOException, InterruptedException {
+    JobConf conf = new JobConf();
+    Path[] paths = new Path[3];
+    File[] files = new File[3];
+    long[] fileLength = new long[3];
+
+    try {
+      for(int i=0;i<3;i++){
+        File dir = new File(outDir.toString());
+        dir.mkdir();
+        files[i] = new File(dir,"testfile"+i);
+        FileWriter fileWriter = new FileWriter(files[i]);
+        fileWriter.flush();
+        fileWriter.close();
+        fileLength[i] = i;
+        paths[i] = new Path(outDir+"/testfile"+i);
+      }
+
+      CombineFileSplit combineFileSplit = new CombineFileSplit(paths, fileLength);
+      TaskAttemptID taskAttemptID = Mockito.mock(TaskAttemptID.class);
+      TaskReporter reporter = Mockito.mock(TaskReporter.class);
+      TaskAttemptContextImpl taskAttemptContext =
+        new TaskAttemptContextImpl(conf, taskAttemptID,reporter);
+
+      CombineFileRecordReader cfrr = new CombineFileRecordReader(combineFileSplit,
+        taskAttemptContext, TextRecordReaderWrapper.class);
+
+      cfrr.initialize(combineFileSplit,taskAttemptContext);
+
+      verify(reporter).progress();
+      Assert.assertFalse(cfrr.nextKeyValue());
+      verify(reporter, times(3)).progress();
+    } finally {
+      FileUtil.fullyDelete(new File(outDir.toString()));
+    }
+  }
+}
+

+ 8 - 0
hadoop-yarn-project/CHANGES.txt

@@ -184,6 +184,10 @@ Release 2.4.0 - UNRELEASED
     YARN-1345. Remove FINAL_SAVING state from YarnApplicationAttemptState
     YARN-1345. Remove FINAL_SAVING state from YarnApplicationAttemptState
     (Zhijie Shen via jianhe)
     (Zhijie Shen via jianhe)
 
 
+    YARN-1676. Modified RM HA handling of user-to-group mappings to
+    be available across RM failover by making using of a remote
+    configuration-provider. (Xuan Gong via vinodkv)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -265,6 +269,10 @@ Release 2.4.0 - UNRELEASED
     YARN-1578. Fixed reading incomplete application attempt and container data
     YARN-1578. Fixed reading incomplete application attempt and container data
     in FileSystemApplicationHistoryStore. (Shinichi Yamashita via zjshen)
     in FileSystemApplicationHistoryStore. (Shinichi Yamashita via zjshen)
 
 
+    YARN-1417. Modified RM to generate container-tokens not at creation time, but
+    at allocation time so as to prevent RM from shelling out containers with
+    expired tokens. (Omkar Vinit Joshi and Jian He via vinodkv)
+
 Release 2.3.1 - UNRELEASED
 Release 2.3.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 9 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java

@@ -381,21 +381,22 @@ public class AdminService extends CompositeService implements
   @Override
   @Override
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
       RefreshUserToGroupsMappingsRequest request)
       RefreshUserToGroupsMappingsRequest request)
-      throws YarnException, StandbyException {
-    UserGroupInformation user = checkAcls("refreshUserToGroupsMappings");
+      throws YarnException, IOException {
+    String argName = "refreshUserToGroupsMappings";
+    UserGroupInformation user = checkAcls(argName);
 
 
-    // TODO (YARN-1459): Revisit handling user-groups on Standby RM
     if (!isRMActive()) {
     if (!isRMActive()) {
-      RMAuditLogger.logFailure(user.getShortUserName(),
-          "refreshUserToGroupsMapping",
+      RMAuditLogger.logFailure(user.getShortUserName(), argName,
           adminAcl.toString(), "AdminService",
           adminAcl.toString(), "AdminService",
           "ResourceManager is not active. Can not refresh user-groups.");
           "ResourceManager is not active. Can not refresh user-groups.");
       throwStandbyException();
       throwStandbyException();
     }
     }
 
 
-    Groups.getUserToGroupsMappingService().refresh();
-    RMAuditLogger.logSuccess(user.getShortUserName(), 
-        "refreshUserToGroupsMappings", "AdminService");
+    Groups.getUserToGroupsMappingService(
+        getConfiguration(getConfig(),
+            YarnConfiguration.CORE_SITE_CONFIGURATION_FILE)).refresh();
+
+    RMAuditLogger.logSuccess(user.getShortUserName(), argName, "AdminService");
 
 
     return recordFactory.newRecordInstance(
     return recordFactory.newRecordInstance(
         RefreshUserToGroupsMappingsResponse.class);
         RefreshUserToGroupsMappingsResponse.class);

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig.Policy;
 import org.apache.hadoop.http.HttpConfig.Policy;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
@@ -198,6 +199,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
       this.conf =
       this.conf =
           this.configurationProvider.getConfiguration(this.conf,
           this.configurationProvider.getConfiguration(this.conf,
               YarnConfiguration.CORE_SITE_CONFIGURATION_FILE);
               YarnConfiguration.CORE_SITE_CONFIGURATION_FILE);
+      // Do refreshUserToGroupsMappings with loaded core-site.xml
+      Groups.getUserToGroupsMappingServiceWithLoadedConfiguration(this.conf)
+          .refresh();
     }
     }
 
 
     // register the handlers for all AlwaysOn services using setupDispatcher().
     // register the handlers for all AlwaysOn services using setupDispatcher().

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java

@@ -345,6 +345,11 @@ public class SchedulerApplicationAttempt {
     for (RMContainer rmContainer : newlyAllocatedContainers) {
     for (RMContainer rmContainer : newlyAllocatedContainers) {
       rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
       rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
           RMContainerEventType.ACQUIRED));
           RMContainerEventType.ACQUIRED));
+      Container container = rmContainer.getContainer();
+      rmContainer.getContainer().setContainerToken(
+        rmContext.getContainerTokenSecretManager().createContainerToken(
+          rmContainer.getContainerId(), container.getNodeId(), getUser(),
+          container.getResource()));
       returnContainerList.add(rmContainer.getContainer());
       returnContainerList.add(rmContainer.getContainer());
     }
     }
     newlyAllocatedContainers.clear();
     newlyAllocatedContainers.clear();

+ 0 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -1292,16 +1292,6 @@ public class LeafQueue implements CSQueue {
     return container;
     return container;
   }
   }
 
 
-  /**
-   * Create <code>ContainerToken</code>, only in secure-mode
-   */
-  Token createContainerToken(
-      FiCaSchedulerApp application, Container container) {
-    return containerTokenSecretManager.createContainerToken(
-        container.getId(), container.getNodeId(),
-        application.getUser(), container.getResource());
-  }
-
   private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node, 
   private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node, 
       FiCaSchedulerApp application, Priority priority, 
       FiCaSchedulerApp application, Priority priority, 
       ResourceRequest request, NodeType type, RMContainer rmContainer) {
       ResourceRequest request, NodeType type, RMContainer rmContainer) {
@@ -1345,14 +1335,6 @@ public class LeafQueue implements CSQueue {
         unreserve(application, priority, node, rmContainer);
         unreserve(application, priority, node, rmContainer);
       }
       }
 
 
-      Token containerToken =
-          createContainerToken(application, container);
-      if (containerToken == null) {
-        // Something went wrong...
-        return Resources.none();
-      }
-      container.setContainerToken(containerToken);
-      
       // Inform the application
       // Inform the application
       RMContainer allocatedContainer = 
       RMContainer allocatedContainer = 
           application.allocate(type, node, priority, request, container);
           application.allocate(type, node, priority, request, container);

+ 1 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java

@@ -151,17 +151,11 @@ public class AppSchedulable extends Schedulable {
     NodeId nodeId = node.getRMNode().getNodeID();
     NodeId nodeId = node.getRMNode().getNodeID();
     ContainerId containerId = BuilderUtils.newContainerId(application
     ContainerId containerId = BuilderUtils.newContainerId(application
         .getApplicationAttemptId(), application.getNewContainerId());
         .getApplicationAttemptId(), application.getNewContainerId());
-    org.apache.hadoop.yarn.api.records.Token containerToken =
-        containerTokenSecretManager.createContainerToken(containerId, nodeId,
-          application.getUser(), capability);
-    if (containerToken == null) {
-      return null; // Try again later.
-    }
 
 
     // Create the container
     // Create the container
     Container container =
     Container container =
         BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
         BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-          .getHttpAddress(), capability, priority, containerToken);
+          .getHttpAddress(), capability, priority, null);
 
 
     return container;
     return container;
   }
   }

+ 1 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -654,20 +654,11 @@ public class FifoScheduler extends AbstractYarnScheduler implements
         NodeId nodeId = node.getRMNode().getNodeID();
         NodeId nodeId = node.getRMNode().getNodeID();
         ContainerId containerId = BuilderUtils.newContainerId(application
         ContainerId containerId = BuilderUtils.newContainerId(application
             .getApplicationAttemptId(), application.getNewContainerId());
             .getApplicationAttemptId(), application.getNewContainerId());
-        Token containerToken = null;
-
-        containerToken =
-            this.rmContext.getContainerTokenSecretManager()
-              .createContainerToken(containerId, nodeId, application.getUser(),
-                capability);
-        if (containerToken == null) {
-          return i; // Try again later.
-        }
 
 
         // Create the container
         // Create the container
         Container container =
         Container container =
             BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
             BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-              .getHttpAddress(), capability, priority, containerToken);
+              .getHttpAddress(), capability, priority, null);
         
         
         // Allocate!
         // Allocate!
         
         

+ 8 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -142,8 +142,15 @@ public class MockRM extends ResourceManager {
   public void waitForState(MockNM nm, ContainerId containerId,
   public void waitForState(MockNM nm, ContainerId containerId,
       RMContainerState containerState) throws Exception {
       RMContainerState containerState) throws Exception {
     RMContainer container = getResourceScheduler().getRMContainer(containerId);
     RMContainer container = getResourceScheduler().getRMContainer(containerId);
-    Assert.assertNotNull("Container shouldn't be null", container);
     int timeoutSecs = 0;
     int timeoutSecs = 0;
+    while(container == null && timeoutSecs++ < 20) {
+      nm.nodeHeartbeat(true);
+      container = getResourceScheduler().getRMContainer(containerId);
+      System.out.println("Waiting for container " + containerId + " to be allocated.");
+      Thread.sleep(100);
+    }
+    Assert.assertNotNull("Container shouldn't be null", container);
+    timeoutSecs = 0;
     while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
     while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
       System.out.println("Container : " + containerId + " State is : "
       System.out.println("Container : " + containerId + " State is : "
           + container.getState() + " Waiting for state : " + containerState);
           + container.getState() + " Waiting for state : " + containerState);

+ 120 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java

@@ -24,10 +24,17 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.GroupMappingServiceProvider;
+import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
@@ -37,6 +44,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.junit.After;
 import org.junit.After;
@@ -366,6 +374,84 @@ public class TestRMAdminService {
         .get("hadoop.proxyuser.test.hosts").contains("test_hosts"));
         .get("hadoop.proxyuser.test.hosts").contains("test_hosts"));
   }
   }
 
 
+  @Test
+  public void testRefreshUserToGroupsMappingsWithLocalConfigurationProvider() {
+    rm = new MockRM(configuration);
+    rm.init(configuration);
+    rm.start();
+    try {
+      rm.adminService
+          .refreshUserToGroupsMappings(RefreshUserToGroupsMappingsRequest
+              .newInstance());
+    } catch (Exception ex) {
+      fail("Using localConfigurationProvider. Should not get any exception.");
+    }
+  }
+
+  @Test
+  public void
+      testRefreshUserToGroupsMappingsWithFileSystemBasedConfigurationProvider()
+          throws IOException, YarnException {
+    configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
+        "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
+    try {
+      rm = new MockRM(configuration);
+      rm.init(configuration);
+      rm.start();
+      fail("Should throw an exception");
+    } catch (Exception ex) {
+      // Expect exception here
+    }
+
+    String user = UserGroupInformation.getCurrentUser().getUserName();
+    List<String> groupWithInit =
+        new ArrayList<String>(Groups.getUserToGroupsMappingService(
+            configuration).getGroups(user));
+
+    // upload default configurations
+    uploadDefaultConfiguration();
+    Configuration conf = new Configuration();
+    conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        MockUnixGroupsMapping.class,
+        GroupMappingServiceProvider.class);
+    uploadConfiguration(conf, "core-site.xml");
+
+    try {
+      rm = new MockRM(configuration);
+      rm.init(configuration);
+      rm.start();
+    } catch (Exception ex) {
+      fail("Should not get any exceptions");
+    }
+
+    // Make sure RM will use the updated GroupMappingServiceProvider
+    List<String> groupBefore =
+        new ArrayList<String>(Groups.getUserToGroupsMappingService(
+            configuration).getGroups(user));
+    Assert.assertTrue(groupBefore.contains("test_group_A")
+        && groupBefore.contains("test_group_B")
+        && groupBefore.contains("test_group_C") && groupBefore.size() == 3);
+    Assert.assertTrue(groupWithInit.size() != groupBefore.size());
+    Assert.assertFalse(groupWithInit.contains("test_group_A")
+        || groupWithInit.contains("test_group_B")
+        || groupWithInit.contains("test_group_C"));
+
+    // update the groups
+    MockUnixGroupsMapping.updateGroups();
+
+    rm.adminService
+        .refreshUserToGroupsMappings(RefreshUserToGroupsMappingsRequest
+            .newInstance());
+    List<String> groupAfter =
+        Groups.getUserToGroupsMappingService(configuration).getGroups(user);
+
+    // should get the updated groups
+    Assert.assertTrue(groupAfter.contains("test_group_D")
+        && groupAfter.contains("test_group_E")
+        && groupAfter.contains("test_group_F") && groupAfter.size() == 3);
+
+  }
+
   private String writeConfigurationXML(Configuration conf, String confXMLName)
   private String writeConfigurationXML(Configuration conf, String confXMLName)
       throws IOException {
       throws IOException {
     DataOutputStream output = null;
     DataOutputStream output = null;
@@ -418,4 +504,38 @@ public class TestRMAdminService {
         .addResource(YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
         .addResource(YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
     uploadConfiguration(hadoopPolicyConf, "hadoop-policy.xml");
     uploadConfiguration(hadoopPolicyConf, "hadoop-policy.xml");
   }
   }
+
+  private static class MockUnixGroupsMapping implements
+      GroupMappingServiceProvider {
+
+    @SuppressWarnings("serial")
+    private static List<String> group = new ArrayList<String>() {{
+      add("test_group_A");
+      add("test_group_B");
+      add("test_group_C");
+    }};
+
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      return group;
+    }
+
+    @Override
+    public void cacheGroupsRefresh() throws IOException {
+      // Do nothing
+    }
+
+    @Override
+    public void cacheGroupsAdd(List<String> groups) throws IOException {
+      // Do nothing
+    }
+
+    public static void updateGroups() {
+      group.clear();
+      group.add("test_group_D");
+      group.add("test_group_E");
+      group.add("test_group_F");
+    }
+  }
+
 }
 }

+ 43 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

@@ -18,11 +18,17 @@
 
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
+import java.util.ArrayList;
+import java.util.List;
+
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -30,6 +36,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.junit.Test;
 import org.junit.Test;
 
 
 
 
@@ -106,4 +115,38 @@ public class TestContainerAllocation {
 
 
     rm.stop();
     rm.stop();
   }
   }
+
+  // This is to test container tokens are generated when the containers are
+  // acquired by the AM, not when the containers are allocated
+  @Test
+  public void testContainerTokenGeneratedOnPullRequest() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+      ResourceScheduler.class);
+    MockRM rm1 = new MockRM(conf);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("127.0.0.1:1234", 8000);
+    RMApp app1 = rm1.submitApp(200);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    // request a container.
+    am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
+    ContainerId containerId2 =
+        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+    rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
+
+    RMContainer container =
+        rm1.getResourceScheduler().getRMContainer(containerId2);
+    // no container token is generated.
+    Assert.assertEquals(containerId2, container.getContainerId());
+    Assert.assertNull(container.getContainer().getContainerToken());
+
+    // acquire the container.
+    List<Container> containers =
+        am1.allocate(new ArrayList<ResourceRequest>(),
+          new ArrayList<ContainerId>()).getAllocatedContainers();
+    Assert.assertEquals(containerId2, containers.get(0).getId());
+    // container token is generated.
+    Assert.assertNotNull(containers.get(0).getContainerToken());
+    rm1.stop();
+  }
 }
 }