Forráskód Böngészése

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 éve
szülő
commit
d8fd23fc46
26 módosított fájl, 535 hozzáadás és 130 törlés
  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.
     (René Nyffenegger via suresh)
 
+    HADOOP-10343. Change info to debug log in LossyRetryInvocationHandler. (arpit)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -337,6 +339,9 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10338. Cannot get the FileStatus of the root inode from the new
     Globber (cmccabe)
 
+    HADOOP-10249. LdapGroupsMapping should trim ldap password read from file.
+    (Dilli Armugam via suresh)
+
 Release 2.3.1 - UNRELEASED
 
   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();
     if (retryCount < this.numToDrop) {
       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");
     } else {
-      LOG.info("retryCount == " + retryCount
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("retryCount == " + retryCount
           + ". It's time to normally process the response");
+      }
       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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -240,4 +241,18 @@ public class 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();
       }
       reader.close();
-      return password.toString();
+      return password.toString().trim();
     } catch (IOException 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) {
         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,
           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
     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 
     closed before method returns. (Ted Yu via junping_du)
 
@@ -275,8 +250,6 @@ Trunk (Unreleased)
     HDFS-5719. FSImage#doRollback() should close prevState before return
     (Ted Yu via brandonli)
 
-    HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
-
     HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager
     (Haohui Mai via brandonli)
 
@@ -376,6 +349,33 @@ Release 2.4.0 - UNRELEASED
     HDFS-5940. Minor cleanups to ShortCircuitReplica, FsDatasetCache, and
     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
 
     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
     dfs.namenode.secondary.https-address in hdfs-default.xml.
     (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
 
   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;
 
-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.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 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.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceStatus;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 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.DFSUtil;
 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.server.common.HdfsServerConstants.NamenodeRole;
 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.startupprogress.StartupProgress;
 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.StandbyException;
 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.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
@@ -183,8 +170,10 @@ public class NameNode implements NameNodeStatusMXBean {
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
     DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_HTTP_ADDRESS_KEY,
+    DFS_NAMENODE_HTTPS_ADDRESS_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+    DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY,
     DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_BACKUP_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
    contributor license agreements.  See the NOTICE file distributed with
@@ -14,8 +16,6 @@
    See the License for the specific language governing permissions and
    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">
 <head>
 <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": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
       {"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 = {
@@ -166,14 +166,29 @@
 
   $('#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
     contributor license agreements.  See the NOTICE file distributed with
@@ -14,8 +16,6 @@
     See the License for the specific language governing permissions and
     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">
   <head>
     <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();
   }
 
+  $(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) {
     return function (jqxhr, text, err) {
       var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
@@ -145,6 +157,7 @@
 
       current_directory = dir;
       $('#directory').val(dir);
+      window.location.hash = dir;
       dust.render('explorer', base.push(d), function(err, out) {
         $('#panel').html(out);
 
@@ -169,7 +182,12 @@
 
     var b = function() { browse_directory($('#directory').val()); };
     $('#btn-nav-directory').click(b);
-    browse_directory('/');
+    var dir = window.location.hash.slice(1);
+    if(dir == "") {
+      window.location.hash = "/";
+    } else {
+      browse_directory(dir);
+    }
   }
 
   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.
     (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
 
   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;
     }
 
+    reporter.progress();
+
     // get a record reader for the idx-th chunk
     try {
       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 long progress;
   protected RecordReader<K, V> curReader;
-  
+
   public void initialize(InputSplit split,
       TaskAttemptContext context) throws IOException, InterruptedException {
     this.split = (CombineFileSplit)split;
@@ -144,6 +144,8 @@ public class CombineFileRecordReader<K, V> extends RecordReader<K, V> {
       return false;
     }
 
+    context.progress();
+
     // get a record reader for the idx-th chunk
     try {
       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
     (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
 
   BUG FIXES
@@ -265,6 +269,10 @@ Release 2.4.0 - UNRELEASED
     YARN-1578. Fixed reading incomplete application attempt and container data
     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
 
   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
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
       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()) {
-      RMAuditLogger.logFailure(user.getShortUserName(),
-          "refreshUserToGroupsMapping",
+      RMAuditLogger.logFailure(user.getShortUserName(), argName,
           adminAcl.toString(), "AdminService",
           "ResourceManager is not active. Can not refresh user-groups.");
       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(
         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.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
@@ -198,6 +199,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
       this.conf =
           this.configurationProvider.getConfiguration(this.conf,
               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().

+ 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) {
       rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
           RMContainerEventType.ACQUIRED));
+      Container container = rmContainer.getContainer();
+      rmContainer.getContainer().setContainerToken(
+        rmContext.getContainerTokenSecretManager().createContainerToken(
+          rmContainer.getContainerId(), container.getNodeId(), getUser(),
+          container.getResource()));
       returnContainerList.add(rmContainer.getContainer());
     }
     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;
   }
 
-  /**
-   * 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, 
       FiCaSchedulerApp application, Priority priority, 
       ResourceRequest request, NodeType type, RMContainer rmContainer) {
@@ -1345,14 +1335,6 @@ public class LeafQueue implements CSQueue {
         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
       RMContainer allocatedContainer = 
           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();
     ContainerId containerId = BuilderUtils.newContainerId(application
         .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
     Container container =
         BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-          .getHttpAddress(), capability, priority, containerToken);
+          .getHttpAddress(), capability, priority, null);
 
     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();
         ContainerId containerId = BuilderUtils.newContainerId(application
             .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
         Container container =
             BuilderUtils.newContainer(containerId, nodeId, node.getRMNode()
-              .getHttpAddress(), capability, priority, containerToken);
+              .getHttpAddress(), capability, priority, null);
         
         // 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,
       RMContainerState containerState) throws Exception {
     RMContainer container = getResourceScheduler().getRMContainer(containerId);
-    Assert.assertNotNull("Container shouldn't be null", container);
     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) {
       System.out.println("Container : " + containerId + " State is : "
           + 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.FileOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 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.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.ProxyUsers;
 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.RefreshServiceAclsRequest;
 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.CapacitySchedulerConfiguration;
 import org.junit.After;
@@ -366,6 +374,84 @@ public class TestRMAdminService {
         .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)
       throws IOException {
     DataOutputStream output = null;
@@ -418,4 +504,38 @@ public class TestRMAdminService {
         .addResource(YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
     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;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import junit.framework.Assert;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.server.resourcemanager.MockAM;
 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.rmapp.RMApp;
 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;
 
 
@@ -106,4 +115,38 @@ public class TestContainerAllocation {
 
     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();
+  }
 }