浏览代码

Merge r1517887 through r1518850 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1519796 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 年之前
父节点
当前提交
336c0344f5
共有 47 个文件被更改,包括 997 次插入164 次删除
  1. 12 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
  3. 6 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
  4. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  5. 19 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java
  6. 2 2
      hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
  7. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElector.java
  8. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java
  9. 17 16
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java
  10. 12 0
      hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
  11. 5 1
      hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java
  12. 4 0
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java
  13. 9 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
  14. 10 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  15. 39 6
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
  16. 12 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  17. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  18. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  19. 77 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  20. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  21. 32 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  22. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  23. 24 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
  24. 29 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  25. 21 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  26. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  27. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
  28. 148 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
  29. 63 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java
  30. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  31. 8 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  32. 47 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
  33. 31 19
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  34. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java
  35. 24 0
      hadoop-yarn-project/CHANGES.txt
  36. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
  37. 46 46
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  38. 21 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java
  39. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
  40. 42 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestLogDumper.java
  41. 5 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  42. 57 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  43. 14 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  44. 16 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
  45. 42 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
  46. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
  47. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

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

@@ -327,6 +327,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9487 Deprecation warnings in Configuration should go to their
     HADOOP-9487 Deprecation warnings in Configuration should go to their
     own log or otherwise be suppressible (Chu Tong via stevel)
     own log or otherwise be suppressible (Chu Tong via stevel)
 
 
+    HADOOP-9889. Refresh the Krb5 configuration when creating a new kdc in
+    Hadoop-MiniKDC (Wei Yan via Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -362,6 +365,9 @@ Release 2.1.1-beta - UNRELEASED
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    HADOOP-9910. proxy server start and stop documentation wrong
+    (Andre Kelpe via harsh)
+
     HADOOP-9446. Support Kerberos SPNEGO for IBM JDK. (Yu Gao via llu)
     HADOOP-9446. Support Kerberos SPNEGO for IBM JDK. (Yu Gao via llu)
  
  
     HADOOP-9787. ShutdownHelper util to shutdown threads and threadpools.
     HADOOP-9787. ShutdownHelper util to shutdown threads and threadpools.
@@ -387,6 +393,9 @@ Release 2.1.1-beta - UNRELEASED
     
     
     HADOOP-9886. Turn warning message in RetryInvocationHandler to debug (arpit)
     HADOOP-9886. Turn warning message in RetryInvocationHandler to debug (arpit)
 
 
+    HADOOP-9906. Move HAZKUtil to o.a.h.util.ZKUtil and make inner-classes
+    public (Karthik Kambatla via Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -423,6 +432,9 @@ Release 2.1.1-beta - UNRELEASED
     HADOOP-9887. globStatus does not correctly handle paths starting with a drive
     HADOOP-9887. globStatus does not correctly handle paths starting with a drive
     spec on Windows. (Chuan Liu via cnauroth)
     spec on Windows. (Chuan Liu via cnauroth)
 
 
+    HADOOP-9894.  Race condition in Shell leads to logged error stream handling
+    exceptions (Arpit Agarwal)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 1 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java

@@ -31,7 +31,7 @@ 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.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
+import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException;
@@ -47,7 +47,6 @@ import org.apache.zookeeper.KeeperException.Code;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 
 /**
 /**
  * 
  * 

+ 6 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java

@@ -36,7 +36,8 @@ import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
 import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
-import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.apache.hadoop.ha.HealthMonitor.State;
 import org.apache.hadoop.ha.HealthMonitor.State;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
@@ -313,18 +314,18 @@ public abstract class ZKFailoverController {
         ZK_SESSION_TIMEOUT_DEFAULT);
         ZK_SESSION_TIMEOUT_DEFAULT);
     // Parse ACLs from configuration.
     // Parse ACLs from configuration.
     String zkAclConf = conf.get(ZK_ACL_KEY, ZK_ACL_DEFAULT);
     String zkAclConf = conf.get(ZK_ACL_KEY, ZK_ACL_DEFAULT);
-    zkAclConf = HAZKUtil.resolveConfIndirection(zkAclConf);
-    List<ACL> zkAcls = HAZKUtil.parseACLs(zkAclConf);
+    zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
+    List<ACL> zkAcls = ZKUtil.parseACLs(zkAclConf);
     if (zkAcls.isEmpty()) {
     if (zkAcls.isEmpty()) {
       zkAcls = Ids.CREATOR_ALL_ACL;
       zkAcls = Ids.CREATOR_ALL_ACL;
     }
     }
     
     
     // Parse authentication from configuration.
     // Parse authentication from configuration.
     String zkAuthConf = conf.get(ZK_AUTH_KEY);
     String zkAuthConf = conf.get(ZK_AUTH_KEY);
-    zkAuthConf = HAZKUtil.resolveConfIndirection(zkAuthConf);
+    zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
     List<ZKAuthInfo> zkAuths;
     List<ZKAuthInfo> zkAuths;
     if (zkAuthConf != null) {
     if (zkAuthConf != null) {
-      zkAuths = HAZKUtil.parseAuth(zkAuthConf);
+      zkAuths = ZKUtil.parseAuth(zkAuthConf);
     } else {
     } else {
       zkAuths = Collections.emptyList();
       zkAuths = Collections.emptyList();
     }
     }

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -515,8 +515,13 @@ abstract public class Shell {
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         LOG.warn("Error while closing the input stream", ioe);
         LOG.warn("Error while closing the input stream", ioe);
       }
       }
-      if (!completed.get()) {
-        errThread.interrupt();
+      try {
+        if (!completed.get()) {
+          errThread.interrupt();
+          errThread.join();
+        }
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted while joining errThread");
       }
       }
       try {
       try {
         errReader.close();
         errReader.close();

+ 19 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAZKUtil.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java

@@ -15,7 +15,7 @@
  * 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.
  */
  */
-package org.apache.hadoop.ha;
+package org.apache.hadoop.util;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
@@ -36,7 +36,7 @@ import com.google.common.io.Files;
  * Utilities for working with ZooKeeper.
  * Utilities for working with ZooKeeper.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class HAZKUtil {
+public class ZKUtil {
   
   
   /**
   /**
    * Parse ACL permission string, partially borrowed from
    * Parse ACL permission string, partially borrowed from
@@ -76,9 +76,10 @@ public class HAZKUtil {
    * <code>sasl:hdfs/host1@MY.DOMAIN:cdrwa,sasl:hdfs/host2@MY.DOMAIN:cdrwa</code>
    * <code>sasl:hdfs/host1@MY.DOMAIN:cdrwa,sasl:hdfs/host2@MY.DOMAIN:cdrwa</code>
    *
    *
    * @return ACL list
    * @return ACL list
-   * @throws HadoopIllegalArgumentException if an ACL is invalid
+   * @throws {@link BadAclFormatException} if an ACL is invalid
    */
    */
-  public static List<ACL> parseACLs(String aclString) {
+  public static List<ACL> parseACLs(String aclString) throws
+      BadAclFormatException {
     List<ACL> acl = Lists.newArrayList();
     List<ACL> acl = Lists.newArrayList();
     if (aclString == null) {
     if (aclString == null) {
       return acl;
       return acl;
@@ -113,8 +114,10 @@ public class HAZKUtil {
    * 
    * 
    * @param authString the comma-separated auth mechanisms
    * @param authString the comma-separated auth mechanisms
    * @return a list of parsed authentications
    * @return a list of parsed authentications
+   * @throws {@link BadAuthFormatException} if the auth format is invalid
    */
    */
-  public static List<ZKAuthInfo> parseAuth(String authString) {
+  public static List<ZKAuthInfo> parseAuth(String authString) throws
+      BadAuthFormatException{
     List<ZKAuthInfo> ret = Lists.newArrayList();
     List<ZKAuthInfo> ret = Lists.newArrayList();
     if (authString == null) {
     if (authString == null) {
       return ret;
       return ret;
@@ -161,7 +164,8 @@ public class HAZKUtil {
   /**
   /**
    * An authentication token passed to ZooKeeper.addAuthInfo
    * An authentication token passed to ZooKeeper.addAuthInfo
    */
    */
-  static class ZKAuthInfo {
+  @InterfaceAudience.Private
+  public static class ZKAuthInfo {
     private final String scheme;
     private final String scheme;
     private final byte[] auth;
     private final byte[] auth;
     
     
@@ -171,29 +175,32 @@ public class HAZKUtil {
       this.auth = auth;
       this.auth = auth;
     }
     }
 
 
-    String getScheme() {
+    public String getScheme() {
       return scheme;
       return scheme;
     }
     }
 
 
-    byte[] getAuth() {
+    public byte[] getAuth() {
       return auth;
       return auth;
     }
     }
   }
   }
 
 
-  static class BadAclFormatException extends HadoopIllegalArgumentException {
+  @InterfaceAudience.Private
+  public static class BadAclFormatException extends
+      HadoopIllegalArgumentException {
     private static final long serialVersionUID = 1L;
     private static final long serialVersionUID = 1L;
 
 
     public BadAclFormatException(String message) {
     public BadAclFormatException(String message) {
       super(message);
       super(message);
     }
     }
   }
   }
-  
-  static class BadAuthFormatException extends HadoopIllegalArgumentException {
+
+  @InterfaceAudience.Private
+  public static class BadAuthFormatException extends
+      HadoopIllegalArgumentException {
     private static final long serialVersionUID = 1L;
     private static final long serialVersionUID = 1L;
 
 
     public BadAuthFormatException(String message) {
     public BadAuthFormatException(String message) {
       super(message);
       super(message);
     }
     }
   }
   }
-
 }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm

@@ -518,7 +518,7 @@ $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start nodemana
   are used with load balancing it should be run on each of them:
   are used with load balancing it should be run on each of them:
 
 
 ----
 ----
-$ $HADOOP_YARN_HOME/bin/yarn start proxyserver --config $HADOOP_CONF_DIR
+$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh start proxyserver --config $HADOOP_CONF_DIR
 ----
 ----
 
 
   Start the MapReduce JobHistory Server with the following command, run on the
   Start the MapReduce JobHistory Server with the following command, run on the
@@ -560,7 +560,7 @@ $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop nodemanag
   balancing it should be run on each of them:
   balancing it should be run on each of them:
 
 
 ----
 ----
-$ $HADOOP_YARN_HOME/bin/yarn stop proxyserver --config $HADOOP_CONF_DIR
+$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh stop proxyserver --config $HADOOP_CONF_DIR
 ----
 ----
 
 
 
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElector.java

@@ -41,7 +41,7 @@ import org.mockito.Mockito;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
-import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
+import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 
 
 public class TestActiveStandbyElector {
 public class TestActiveStandbyElector {

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java

@@ -28,7 +28,7 @@ import java.util.UUID;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
 import org.apache.hadoop.ha.ActiveStandbyElector.State;
 import org.apache.hadoop.ha.ActiveStandbyElector.State;
-import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
+import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;

+ 17 - 16
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAZKUtil.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java

@@ -15,7 +15,7 @@
  * 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.
  */
  */
-package org.apache.hadoop.ha;
+package org.apache.hadoop.util;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
@@ -24,8 +24,9 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.ha.HAZKUtil.BadAclFormatException;
-import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.ZKUtil.BadAclFormatException;
+import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.apache.zookeeper.ZooDefs.Perms;
 import org.apache.zookeeper.ZooDefs.Perms;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.ACL;
 import org.junit.Test;
 import org.junit.Test;
@@ -33,9 +34,9 @@ import org.junit.Test;
 import com.google.common.base.Charsets;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import com.google.common.io.Files;
 
 
-public class TestHAZKUtil {
+public class TestZKUtil {
   private static final String TEST_ROOT_DIR = System.getProperty(
   private static final String TEST_ROOT_DIR = System.getProperty(
-      "test.build.data", "/tmp") + "/TestHAZKUtil";
+      "test.build.data", "/tmp") + "/TestZKUtil";
   private static final File TEST_FILE = new File(TEST_ROOT_DIR,
   private static final File TEST_FILE = new File(TEST_ROOT_DIR,
       "test-file");
       "test-file");
   
   
@@ -45,13 +46,13 @@ public class TestHAZKUtil {
 
 
   @Test
   @Test
   public void testEmptyACL() {
   public void testEmptyACL() {
-    List<ACL> result = HAZKUtil.parseACLs("");
+    List<ACL> result = ZKUtil.parseACLs("");
     assertTrue(result.isEmpty());
     assertTrue(result.isEmpty());
   }
   }
   
   
   @Test
   @Test
   public void testNullACL() {
   public void testNullACL() {
-    List<ACL> result = HAZKUtil.parseACLs(null);
+    List<ACL> result = ZKUtil.parseACLs(null);
     assertTrue(result.isEmpty());
     assertTrue(result.isEmpty());
   }
   }
   
   
@@ -67,7 +68,7 @@ public class TestHAZKUtil {
   
   
   private static void badAcl(String acls, String expectedErr) {
   private static void badAcl(String acls, String expectedErr) {
     try {
     try {
-      HAZKUtil.parseACLs(acls);
+      ZKUtil.parseACLs(acls);
       fail("Should have failed to parse '" + acls + "'");
       fail("Should have failed to parse '" + acls + "'");
     } catch (BadAclFormatException e) {
     } catch (BadAclFormatException e) {
       assertEquals(expectedErr, e.getMessage());
       assertEquals(expectedErr, e.getMessage());
@@ -76,7 +77,7 @@ public class TestHAZKUtil {
 
 
   @Test
   @Test
   public void testGoodACLs() {
   public void testGoodACLs() {
-    List<ACL> result = HAZKUtil.parseACLs(
+    List<ACL> result = ZKUtil.parseACLs(
         "sasl:hdfs/host1@MY.DOMAIN:cdrwa, sasl:hdfs/host2@MY.DOMAIN:ca");
         "sasl:hdfs/host1@MY.DOMAIN:cdrwa, sasl:hdfs/host2@MY.DOMAIN:ca");
     ACL acl0 = result.get(0);
     ACL acl0 = result.get(0);
     assertEquals(Perms.CREATE | Perms.DELETE | Perms.READ |
     assertEquals(Perms.CREATE | Perms.DELETE | Perms.READ |
@@ -92,19 +93,19 @@ public class TestHAZKUtil {
   
   
   @Test
   @Test
   public void testEmptyAuth() {
   public void testEmptyAuth() {
-    List<ZKAuthInfo> result = HAZKUtil.parseAuth("");
+    List<ZKAuthInfo> result = ZKUtil.parseAuth("");
     assertTrue(result.isEmpty());
     assertTrue(result.isEmpty());
   }
   }
   
   
   @Test
   @Test
   public void testNullAuth() {
   public void testNullAuth() {
-    List<ZKAuthInfo> result = HAZKUtil.parseAuth(null);
+    List<ZKAuthInfo> result = ZKUtil.parseAuth(null);
     assertTrue(result.isEmpty());
     assertTrue(result.isEmpty());
   }
   }
   
   
   @Test
   @Test
   public void testGoodAuths() {
   public void testGoodAuths() {
-    List<ZKAuthInfo> result = HAZKUtil.parseAuth(
+    List<ZKAuthInfo> result = ZKUtil.parseAuth(
         "scheme:data,\n   scheme2:user:pass");
         "scheme:data,\n   scheme2:user:pass");
     assertEquals(2, result.size());
     assertEquals(2, result.size());
     ZKAuthInfo auth0 = result.get(0);
     ZKAuthInfo auth0 = result.get(0);
@@ -118,16 +119,16 @@ public class TestHAZKUtil {
   
   
   @Test
   @Test
   public void testConfIndirection() throws IOException {
   public void testConfIndirection() throws IOException {
-    assertNull(HAZKUtil.resolveConfIndirection(null));
-    assertEquals("x", HAZKUtil.resolveConfIndirection("x"));
+    assertNull(ZKUtil.resolveConfIndirection(null));
+    assertEquals("x", ZKUtil.resolveConfIndirection("x"));
     
     
     TEST_FILE.getParentFile().mkdirs();
     TEST_FILE.getParentFile().mkdirs();
     Files.write("hello world", TEST_FILE, Charsets.UTF_8);
     Files.write("hello world", TEST_FILE, Charsets.UTF_8);
-    assertEquals("hello world", HAZKUtil.resolveConfIndirection(
+    assertEquals("hello world", ZKUtil.resolveConfIndirection(
         "@" + TEST_FILE.getAbsolutePath()));
         "@" + TEST_FILE.getAbsolutePath()));
     
     
     try {
     try {
-      HAZKUtil.resolveConfIndirection("@" + BOGUS_FILE);
+      ZKUtil.resolveConfIndirection("@" + BOGUS_FILE);
       fail("Did not throw for non-existent file reference");
       fail("Did not throw for non-existent file reference");
     } catch (FileNotFoundException fnfe) {
     } catch (FileNotFoundException fnfe) {
       assertTrue(fnfe.getMessage().startsWith(BOGUS_FILE));
       assertTrue(fnfe.getMessage().startsWith(BOGUS_FILE));

+ 12 - 0
hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java

@@ -59,6 +59,7 @@ import java.io.FileReader;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.io.StringReader;
 import java.io.StringReader;
+import java.lang.reflect.Method;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.ServerSocket;
 import java.net.ServerSocket;
 import java.text.MessageFormat;
 import java.text.MessageFormat;
@@ -432,6 +433,17 @@ public class MiniKdc {
 
 
     System.setProperty("sun.security.krb5.debug", conf.getProperty(DEBUG,
     System.setProperty("sun.security.krb5.debug", conf.getProperty(DEBUG,
             "false"));
             "false"));
+
+    // refresh the config
+    Class<?> classRef;
+    if (System.getProperty("java.vendor").contains("IBM")) {
+      classRef = Class.forName("com.ibm.security.krb5.internal.Config");
+    } else {
+      classRef = Class.forName("sun.security.krb5.Config");
+    }
+    Method refreshMethod = classRef.getMethod("refresh", new Class[0]);
+    refreshMethod.invoke(classRef, new Object[0]);
+
     LOG.info("MiniKdc listening at port: {}", getPort());
     LOG.info("MiniKdc listening at port: {}", getPort());
     LOG.info("MiniKdc setting JVM krb5.conf to: {}",
     LOG.info("MiniKdc setting JVM krb5.conf to: {}",
             krb5conf.getAbsolutePath());
             krb5conf.getAbsolutePath());

+ 5 - 1
hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java

@@ -30,7 +30,11 @@ import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.login.LoginContext;
 import java.io.File;
 import java.io.File;
 import java.security.Principal;
 import java.security.Principal;
-import java.util.*;
+import java.util.Set;
+import java.util.Map;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Arrays;
 
 
 public class TestMiniKdc extends KerberosSecurityTestcase {
 public class TestMiniKdc extends KerberosSecurityTestcase {
 
 

+ 4 - 0
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Constant.java

@@ -201,4 +201,8 @@ public class Nfs3Constant {
   public static final String EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY = "hdfs.nfs.exports.cache.expirytime.millis";
   public static final String EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY = "hdfs.nfs.exports.cache.expirytime.millis";
   public static final long EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 15 * 60 * 1000; // 15 min
   public static final long EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 15 * 60 * 1000; // 15 min
 
 
+  public static final String FILE_DUMP_DIR_KEY = "dfs.nfs3.dump.dir";
+  public static final String FILE_DUMP_DIR_DEFAULT = "/tmp/.hdfs-nfs";
+  public static final String ENABLE_FILE_DUMP_KEY = "dfs.nfs3.enableDump";
+  public static final boolean ENABLE_FILE_DUMP_DEFAULT = true;
 }
 }

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java

@@ -126,6 +126,9 @@ class OpenFileCtx {
     nonSequentialWriteInMemory = 0;
     nonSequentialWriteInMemory = 0;
     this.dumpFilePath = dumpFilePath;  
     this.dumpFilePath = dumpFilePath;  
     enabledDump = dumpFilePath == null ? false: true;
     enabledDump = dumpFilePath == null ? false: true;
+    nextOffset = latestAttr.getSize();
+    assert(nextOffset == this.fos.getPos());
+
     ctxLock = new ReentrantLock(true);
     ctxLock = new ReentrantLock(true);
   }
   }
 
 
@@ -685,12 +688,14 @@ class OpenFileCtx {
 
 
     try {
     try {
       fos.write(data, 0, count);
       fos.write(data, 0, count);
-
-      if (fos.getPos() != (offset + count)) {
+      
+      long flushedOffset = getFlushedOffset();
+      if (flushedOffset != (offset + count)) {
         throw new IOException("output stream is out of sync, pos="
         throw new IOException("output stream is out of sync, pos="
-            + fos.getPos() + " and nextOffset should be" + (offset + count));
+            + flushedOffset + " and nextOffset should be"
+            + (offset + count));
       }
       }
-      nextOffset = fos.getPos();
+      nextOffset = flushedOffset;
 
 
       // Reduce memory occupation size if request was allowed dumped
       // Reduce memory occupation size if request was allowed dumped
       if (writeCtx.getDataState() == DataState.ALLOW_DUMP) {
       if (writeCtx.getDataState() == DataState.ALLOW_DUMP) {

+ 10 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
@@ -123,7 +124,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
 
 
   private final Configuration config = new Configuration();
   private final Configuration config = new Configuration();
   private final WriteManager writeManager;
   private final WriteManager writeManager;
-  private final IdUserGroup iug;// = new IdUserGroup();
+  private final IdUserGroup iug;
   private final DFSClientCache clientCache;
   private final DFSClientCache clientCache;
 
 
   private final NfsExports exports;
   private final NfsExports exports;
@@ -161,10 +162,14 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         DFSConfigKeys.DFS_REPLICATION_DEFAULT);
         DFSConfigKeys.DFS_REPLICATION_DEFAULT);
     blockSize = config.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
     blockSize = config.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
         DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
         DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
-    bufferSize = config.getInt("io.file.buffer.size", 4096);
+    bufferSize = config.getInt(
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+        CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
     
     
-    writeDumpDir = config.get("dfs.nfs3.dump.dir", "/tmp/.hdfs-nfs");    
-    boolean enableDump = config.getBoolean("dfs.nfs3.enableDump", true);
+    writeDumpDir = config.get(Nfs3Constant.FILE_DUMP_DIR_KEY,
+        Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
+    boolean enableDump = config.getBoolean(Nfs3Constant.ENABLE_FILE_DUMP_KEY,
+        Nfs3Constant.ENABLE_FILE_DUMP_DEFAULT);
     if (!enableDump) {
     if (!enableDump) {
       writeDumpDir = null;
       writeDumpDir = null;
     } else {
     } else {
@@ -1112,6 +1117,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     }
   }
   }
 
 
+  @Override
   public SYMLINK3Response symlink(XDR xdr, RpcAuthSys authSys,
   public SYMLINK3Response symlink(XDR xdr, RpcAuthSys authSys,
       InetAddress client) {
       InetAddress client) {
     return new SYMLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
     return new SYMLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);

+ 39 - 6
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java

@@ -25,7 +25,9 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.nfs.NfsFileType;
 import org.apache.hadoop.nfs.NfsFileType;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
@@ -48,6 +50,7 @@ import com.google.common.collect.Maps;
 public class WriteManager {
 public class WriteManager {
   public static final Log LOG = LogFactory.getLog(WriteManager.class);
   public static final Log LOG = LogFactory.getLog(WriteManager.class);
 
 
+  private final Configuration config;
   private final IdUserGroup iug;
   private final IdUserGroup iug;
   private final ConcurrentMap<FileHandle, OpenFileCtx> openFileMap = Maps
   private final ConcurrentMap<FileHandle, OpenFileCtx> openFileMap = Maps
       .newConcurrentMap();
       .newConcurrentMap();
@@ -76,6 +79,7 @@ public class WriteManager {
 
 
   WriteManager(IdUserGroup iug, final Configuration config) {
   WriteManager(IdUserGroup iug, final Configuration config) {
     this.iug = iug;
     this.iug = iug;
+    this.config = config;
     
     
     streamTimeout = config.getLong("dfs.nfs3.stream.timeout",
     streamTimeout = config.getLong("dfs.nfs3.stream.timeout",
         DEFAULT_STREAM_TIMEOUT);
         DEFAULT_STREAM_TIMEOUT);
@@ -129,12 +133,41 @@ public class WriteManager {
     OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
     OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
     if (openFileCtx == null) {
     if (openFileCtx == null) {
       LOG.info("No opened stream for fileId:" + fileHandle.getFileId());
       LOG.info("No opened stream for fileId:" + fileHandle.getFileId());
-      WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), preOpAttr);
-      WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
-          fileWcc, count, request.getStableHow(),
-          Nfs3Constant.WRITE_COMMIT_VERF);
-      Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
-      return;
+
+      String fileIdPath = Nfs3Utils.getFileIdPath(fileHandle.getFileId());
+      HdfsDataOutputStream fos = null;
+      Nfs3FileAttributes latestAttr = null;
+      try {
+        int bufferSize = config.getInt(
+            CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+            CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+        
+        fos = dfsClient.append(fileIdPath, bufferSize, null, null);
+
+        latestAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
+      } catch (IOException e) {
+        LOG.error("Can't apapend to file:" + fileIdPath + ", error:" + e);
+        if (fos != null) {
+          fos.close();
+        }
+        WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr),
+            preOpAttr);
+        WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
+            fileWcc, count, request.getStableHow(),
+            Nfs3Constant.WRITE_COMMIT_VERF);
+        Nfs3Utils.writeChannel(channel, response.send(new XDR(), xid));
+        return;
+      }
+
+      // Add open stream
+      String writeDumpDir = config.get(Nfs3Constant.FILE_DUMP_DIR_KEY,
+          Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
+      openFileCtx = new OpenFileCtx(fos, latestAttr, writeDumpDir + "/"
+          + fileHandle.getFileId());
+      addOpenFileStream(fileHandle, openFileCtx);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("opened stream for file:" + fileHandle.getFileId());
+      }
     }
     }
 
 
     // Add write into the async job queue
     // Add write into the async job queue

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

@@ -307,6 +307,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-4947 Add NFS server export table to control export by hostname or
     HDFS-4947 Add NFS server export table to control export by hostname or
     IP range (Jing Zhao via brandonli)
     IP range (Jing Zhao via brandonli)
 
 
+    HDFS-5078 Support file append in NFSv3 gateway to enable data streaming
+    to HDFS (brandonli)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may
@@ -333,6 +336,12 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5045. Add more unit tests for retry cache to cover all AtMostOnce 
     HDFS-5045. Add more unit tests for retry cache to cover all AtMostOnce 
     methods. (jing9)
     methods. (jing9)
 
 
+    HDFS-3245. Add metrics and web UI for cluster version summary. (Ravi
+    Prakash via kihwal)
+
+    HDFS-5128. Allow multiple net interfaces to be used with HA namenode RPC
+    server. (kihwal)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -389,6 +398,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5124. DelegationTokenSecretManager#retrievePassword can cause deadlock 
     HDFS-5124. DelegationTokenSecretManager#retrievePassword can cause deadlock 
     in NameNode. (Daryn Sharp via jing9)
     in NameNode. (Daryn Sharp via jing9)
 
 
+    HDFS-5132. Deadlock in NameNode between SafeModeMonitor#run and 
+    DatanodeManager#handleHeartbeat. (kihwal)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -104,7 +104,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTP_PORT_DEFAULT;
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTP_PORT_DEFAULT;
   public static final String  DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
   public static final String  DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
+  public static final String  DFS_NAMENODE_RPC_BIND_HOST_KEY = "dfs.namenode.rpc-bind-host";
   public static final String  DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.servicerpc-address";
   public static final String  DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.servicerpc-address";
+  public static final String  DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY = "dfs.namenode.servicerpc-bind-host";
   public static final String  DFS_NAMENODE_MAX_OBJECTS_KEY = "dfs.namenode.max.objects";
   public static final String  DFS_NAMENODE_MAX_OBJECTS_KEY = "dfs.namenode.max.objects";
   public static final long    DFS_NAMENODE_MAX_OBJECTS_DEFAULT = 0;
   public static final long    DFS_NAMENODE_MAX_OBJECTS_DEFAULT = 0;
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_KEY = "dfs.namenode.safemode.extension";
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_KEY = "dfs.namenode.safemode.extension";

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -47,6 +47,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private long lastUpdate;
   private long lastUpdate;
   private int xceiverCount;
   private int xceiverCount;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String location = NetworkTopology.DEFAULT_RACK;
+  private String softwareVersion;
   
   
   // Datanode administrative states
   // Datanode administrative states
   public enum AdminStates {
   public enum AdminStates {
@@ -383,4 +384,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
     // by DatanodeID
     // by DatanodeID
     return (this == obj) || super.equals(obj);
     return (this == obj) || super.equals(obj);
   }
   }
+
+  public String getSoftwareVersion() {
+    return softwareVersion;
+  }
+
+  public void setSoftwareVersion(String softwareVersion) {
+    this.softwareVersion = softwareVersion;
+  }
 }
 }

+ 77 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -26,6 +26,7 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.NavigableMap;
@@ -165,6 +166,14 @@ public class DatanodeManager {
    * according to the NetworkTopology.
    * according to the NetworkTopology.
    */
    */
   private boolean hasClusterEverBeenMultiRack = false;
   private boolean hasClusterEverBeenMultiRack = false;
+
+  /**
+   * The number of datanodes for each software version. This list should change
+   * during rolling upgrades.
+   * Software version -> Number of datanodes with this version
+   */
+  private HashMap<String, Integer> datanodesSoftwareVersions =
+    new HashMap<String, Integer>(4, 0.75f);
   
   
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
       final Configuration conf) throws IOException {
       final Configuration conf) throws IOException {
@@ -470,6 +479,7 @@ public class DatanodeManager {
     heartbeatManager.removeDatanode(nodeInfo);
     heartbeatManager.removeDatanode(nodeInfo);
     blockManager.removeBlocksAssociatedTo(nodeInfo);
     blockManager.removeBlocksAssociatedTo(nodeInfo);
     networktopology.remove(nodeInfo);
     networktopology.remove(nodeInfo);
+    decrementVersionCount(nodeInfo.getSoftwareVersion());
 
 
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("remove datanode " + nodeInfo);
       LOG.debug("remove datanode " + nodeInfo);
@@ -552,6 +562,61 @@ public class DatanodeManager {
     }
     }
   }
   }
 
 
+  private void incrementVersionCount(String version) {
+    if (version == null) {
+      return;
+    }
+    synchronized(datanodeMap) {
+      Integer count = this.datanodesSoftwareVersions.get(version);
+      count = count == null ? 1 : count + 1;
+      this.datanodesSoftwareVersions.put(version, count);
+    }
+  }
+
+  private void decrementVersionCount(String version) {
+    if (version == null) {
+      return;
+    }
+    synchronized(datanodeMap) {
+      Integer count = this.datanodesSoftwareVersions.get(version);
+      if(count != null) {
+        if(count > 1) {
+          this.datanodesSoftwareVersions.put(version, count-1);
+        } else {
+          this.datanodesSoftwareVersions.remove(version);
+        }
+      }
+    }
+  }
+
+  private boolean shouldCountVersion(DatanodeDescriptor node) {
+    return node.getSoftwareVersion() != null && node.isAlive &&
+      !isDatanodeDead(node);
+  }
+
+  private void countSoftwareVersions() {
+    synchronized(datanodeMap) {
+      HashMap<String, Integer> versionCount = new HashMap<String, Integer>();
+      for(DatanodeDescriptor dn: datanodeMap.values()) {
+        // Check isAlive too because right after removeDatanode(),
+        // isDatanodeDead() is still true 
+        if(shouldCountVersion(dn))
+        {
+          Integer num = versionCount.get(dn.getSoftwareVersion());
+          num = num == null ? 1 : num+1;
+          versionCount.put(dn.getSoftwareVersion(), num);
+        }
+      }
+      this.datanodesSoftwareVersions = versionCount;
+    }
+  }
+
+  public HashMap<String, Integer> getDatanodesSoftwareVersions() {
+    synchronized(datanodeMap) {
+      return new HashMap<String, Integer> (this.datanodesSoftwareVersions);
+    }
+  }
+
   /* Resolve a node's network location */
   /* Resolve a node's network location */
   private String resolveNetworkLocation (DatanodeID node) {
   private String resolveNetworkLocation (DatanodeID node) {
     List<String> names = new ArrayList<String>(1);
     List<String> names = new ArrayList<String>(1);
@@ -751,21 +816,28 @@ public class DatanodeManager {
         try {
         try {
           // update cluster map
           // update cluster map
           getNetworkTopology().remove(nodeS);
           getNetworkTopology().remove(nodeS);
+          if(shouldCountVersion(nodeS)) {
+            decrementVersionCount(nodeS.getSoftwareVersion());
+          }
           nodeS.updateRegInfo(nodeReg);
           nodeS.updateRegInfo(nodeReg);
+
+          nodeS.setSoftwareVersion(nodeReg.getSoftwareVersion());
           nodeS.setDisallowed(false); // Node is in the include list
           nodeS.setDisallowed(false); // Node is in the include list
-          
+
           // resolve network location
           // resolve network location
           nodeS.setNetworkLocation(resolveNetworkLocation(nodeS));
           nodeS.setNetworkLocation(resolveNetworkLocation(nodeS));
           getNetworkTopology().add(nodeS);
           getNetworkTopology().add(nodeS);
             
             
           // also treat the registration message as a heartbeat
           // also treat the registration message as a heartbeat
           heartbeatManager.register(nodeS);
           heartbeatManager.register(nodeS);
+          incrementVersionCount(nodeS.getSoftwareVersion());
           checkDecommissioning(nodeS);
           checkDecommissioning(nodeS);
           success = true;
           success = true;
         } finally {
         } finally {
           if (!success) {
           if (!success) {
             removeDatanode(nodeS);
             removeDatanode(nodeS);
             wipeDatanode(nodeS);
             wipeDatanode(nodeS);
+            countSoftwareVersions();
           }
           }
         }
         }
         return;
         return;
@@ -789,6 +861,7 @@ public class DatanodeManager {
       try {
       try {
         nodeDescr.setNetworkLocation(resolveNetworkLocation(nodeDescr));
         nodeDescr.setNetworkLocation(resolveNetworkLocation(nodeDescr));
         networktopology.add(nodeDescr);
         networktopology.add(nodeDescr);
+        nodeDescr.setSoftwareVersion(nodeReg.getSoftwareVersion());
   
   
         // register new datanode
         // register new datanode
         addDatanode(nodeDescr);
         addDatanode(nodeDescr);
@@ -799,10 +872,12 @@ public class DatanodeManager {
         // because its is done when the descriptor is created
         // because its is done when the descriptor is created
         heartbeatManager.addDatanode(nodeDescr);
         heartbeatManager.addDatanode(nodeDescr);
         success = true;
         success = true;
+        incrementVersionCount(nodeReg.getSoftwareVersion());
       } finally {
       } finally {
         if (!success) {
         if (!success) {
           removeDatanode(nodeDescr);
           removeDatanode(nodeDescr);
           wipeDatanode(nodeDescr);
           wipeDatanode(nodeDescr);
+          countSoftwareVersions();
         }
         }
       }
       }
     } catch (InvalidTopologyException e) {
     } catch (InvalidTopologyException e) {
@@ -824,6 +899,7 @@ public class DatanodeManager {
     namesystem.writeLock();
     namesystem.writeLock();
     try {
     try {
       refreshDatanodes();
       refreshDatanodes();
+      countSoftwareVersions();
     } finally {
     } finally {
       namesystem.writeUnlock();
       namesystem.writeUnlock();
     }
     }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -359,6 +359,7 @@ class ClusterJspHelper {
       nn.httpAddress = httpAddress;
       nn.httpAddress = httpAddress;
       getLiveNodeCount(getProperty(props, "LiveNodes").getValueAsText(), nn);
       getLiveNodeCount(getProperty(props, "LiveNodes").getValueAsText(), nn);
       getDeadNodeCount(getProperty(props, "DeadNodes").getValueAsText(), nn);
       getDeadNodeCount(getProperty(props, "DeadNodes").getValueAsText(), nn);
+      nn.softwareVersion = getProperty(props, "SoftwareVersion").getTextValue();
       return nn;
       return nn;
     }
     }
     
     
@@ -596,6 +597,7 @@ class ClusterJspHelper {
         toXmlItemBlockWithLink(doc, nn.deadDatanodeCount + " (" +
         toXmlItemBlockWithLink(doc, nn.deadDatanodeCount + " (" +
           nn.deadDecomCount + ")", nn.httpAddress+"/dfsnodelist.jsp?whatNodes=DEAD"
           nn.deadDecomCount + ")", nn.httpAddress+"/dfsnodelist.jsp?whatNodes=DEAD"
           , "Dead Datanode (Decommissioned)");
           , "Dead Datanode (Decommissioned)");
+        toXmlItemBlock(doc, "Software Version", nn.softwareVersion);
         doc.endTag(); // node
         doc.endTag(); // node
       }
       }
       doc.endTag(); // namenodes
       doc.endTag(); // namenodes
@@ -624,6 +626,7 @@ class ClusterJspHelper {
     int deadDatanodeCount = 0;
     int deadDatanodeCount = 0;
     int deadDecomCount = 0;
     int deadDecomCount = 0;
     String httpAddress = null;
     String httpAddress = null;
+    String softwareVersion = "";
   }
   }
 
 
   /**
   /**

+ 32 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4804,7 +4804,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
      */
      */
     @Override
     @Override
     public void run() {
     public void run() {
-      while (fsRunning && (safeMode != null && !safeMode.canLeave())) {
+      while (fsRunning) {
+        writeLock();
+        try {
+          if (safeMode == null) { // Not in safe mode.
+            break;
+          }
+          if (safeMode.canLeave()) {
+            // Leave safe mode.
+            safeMode.leave();
+            break;
+          }
+        } finally {
+          writeUnlock();
+        }
+
         try {
         try {
           Thread.sleep(recheckInterval);
           Thread.sleep(recheckInterval);
         } catch (InterruptedException ie) {
         } catch (InterruptedException ie) {
@@ -4813,9 +4827,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       }
       if (!fsRunning) {
       if (!fsRunning) {
         LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread");
         LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread");
-      } else {
-        // leave safe mode and stop the monitor
-        leaveSafeMode();
       }
       }
       smmthread = null;
       smmthread = null;
     }
     }
@@ -6226,6 +6237,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
       innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
       innerinfo.put("capacity", node.getCapacity());
       innerinfo.put("capacity", node.getCapacity());
       innerinfo.put("numBlocks", node.numBlocks());
       innerinfo.put("numBlocks", node.numBlocks());
+      innerinfo.put("version", node.getSoftwareVersion());
       info.put(node.getHostName(), innerinfo);
       info.put(node.getHostName(), innerinfo);
     }
     }
     return JSON.toString(info);
     return JSON.toString(info);
@@ -6437,6 +6449,22 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return JSON.toString(list);
     return JSON.toString(list);
   }
   }
 
 
+  @Override  //NameNodeMXBean
+  public int getDistinctVersionCount() {
+    return blockManager.getDatanodeManager().getDatanodesSoftwareVersions()
+      .size();
+  }
+
+  @Override  //NameNodeMXBean
+  public Map<String, Integer> getDistinctVersions() {
+    return blockManager.getDatanodeManager().getDatanodesSoftwareVersions();
+  }
+
+  @Override  //NameNodeMXBean
+  public String getSoftwareVersion() {
+    return VersionInfo.getVersion();
+  }
+
   /**
   /**
    * Verifies that the given identifier and password are valid and match.
    * Verifies that the given identifier and password are valid and match.
    * @param identifier Token identifier.
    * @param identifier Token identifier.

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -166,12 +166,14 @@ public class NameNode implements NameNodeStatusMXBean {
    */
    */
   public static final String[] NAMENODE_SPECIFIC_KEYS = {
   public static final String[] NAMENODE_SPECIFIC_KEYS = {
     DFS_NAMENODE_RPC_ADDRESS_KEY,
     DFS_NAMENODE_RPC_ADDRESS_KEY,
+    DFS_NAMENODE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_NAME_DIR_KEY,
     DFS_NAMENODE_NAME_DIR_KEY,
     DFS_NAMENODE_EDITS_DIR_KEY,
     DFS_NAMENODE_EDITS_DIR_KEY,
     DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
     DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+    DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY,
     DFS_NAMENODE_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
@@ -387,6 +389,28 @@ public class NameNode implements NameNodeStatusMXBean {
     return getAddress(conf);
     return getAddress(conf);
   }
   }
   
   
+  /** Given a configuration get the bind host of the service rpc server
+   *  If the bind host is not configured returns null.
+   */
+  protected String getServiceRpcServerBindHost(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY);
+    if (addr == null || addr.isEmpty()) {
+      return null;
+    }
+    return addr;
+  }
+
+  /** Given a configuration get the bind host of the client rpc server
+   *  If the bind host is not configured returns null.
+   */
+  protected String getRpcServerBindHost(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_NAMENODE_RPC_BIND_HOST_KEY);
+    if (addr == null || addr.isEmpty()) {
+      return null;
+    }
+    return addr;
+  }
+   
   /**
   /**
    * Modifies the configuration passed to contain the service rpc address setting
    * Modifies the configuration passed to contain the service rpc address setting
    */
    */

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

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 
 
@@ -33,7 +35,13 @@ public interface NameNodeMXBean {
    * @return the version
    * @return the version
    */
    */
   public String getVersion();
   public String getVersion();
-  
+
+  /**
+   * Get the version of software running on the Namenode
+   * @return a string representing the version
+   */
+  public String getSoftwareVersion();
+
   /**
   /**
    * Gets the used space by data nodes.
    * Gets the used space by data nodes.
    * 
    * 
@@ -215,4 +223,19 @@ public interface NameNodeMXBean {
    * @return the list of corrupt files, as a JSON string.
    * @return the list of corrupt files, as a JSON string.
    */
    */
   public String getCorruptFiles();
   public String getCorruptFiles();
+
+  /**
+   * Get the number of distinct versions of live datanodes
+   * 
+   * @return the number of distinct versions of live datanodes
+   */
+  public int getDistinctVersionCount();
+
+  /**
+   * Get the number of live datanodes for each distinct versions
+   * 
+   * @return the number of live datanodes for each distinct versions
+   */
+  public Map<String, Integer> getDistinctVersions();
+  
 }
 }

+ 29 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -137,6 +137,7 @@ import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTrans
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 import org.apache.hadoop.util.VersionUtil;
 
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.BlockingService;
 
 
 /**
 /**
@@ -219,6 +220,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     if (serviceRpcAddr != null) {
     if (serviceRpcAddr != null) {
+      String bindHost = nn.getServiceRpcServerBindHost(conf);
+      if (bindHost == null) {
+        bindHost = serviceRpcAddr.getHostName();
+      }
+      LOG.info("Service RPC server is binding to " + bindHost + ":" +
+          serviceRpcAddr.getPort());
+
       int serviceHandlerCount =
       int serviceHandlerCount =
         conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
         conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
                     DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
                     DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
@@ -226,7 +234,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
           .setProtocol(
           .setProtocol(
               org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
               org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
           .setInstance(clientNNPbService)
           .setInstance(clientNNPbService)
-          .setBindAddress(serviceRpcAddr.getHostName())
+          .setBindAddress(bindHost)
           .setPort(serviceRpcAddr.getPort())
           .setPort(serviceRpcAddr.getPort())
           .setNumHandlers(serviceHandlerCount)
           .setNumHandlers(serviceHandlerCount)
           .setVerbose(false)
           .setVerbose(false)
@@ -247,7 +255,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
       DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
       DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
           getUserMappingService, serviceRpcServer);
           getUserMappingService, serviceRpcServer);
   
   
-      serviceRPCAddress = serviceRpcServer.getListenerAddress();
+      // Update the address with the correct port
+      InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
+      serviceRPCAddress = new InetSocketAddress(
+            serviceRpcAddr.getHostName(), listenAddr.getPort());
       nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
       nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
     } else {
     } else {
       serviceRpcServer = null;
       serviceRpcServer = null;
@@ -255,11 +266,17 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     }
 
 
     InetSocketAddress rpcAddr = nn.getRpcServerAddress(conf);
     InetSocketAddress rpcAddr = nn.getRpcServerAddress(conf);
+    String bindHost = nn.getRpcServerBindHost(conf);
+    if (bindHost == null) {
+      bindHost = rpcAddr.getHostName();
+    }
+    LOG.info("RPC server is binding to " + bindHost + ":" + rpcAddr.getPort());
+
     clientRpcServer = new RPC.Builder(conf)
     clientRpcServer = new RPC.Builder(conf)
         .setProtocol(
         .setProtocol(
             org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
             org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
         .setInstance(clientNNPbService)
         .setInstance(clientNNPbService)
-        .setBindAddress(rpcAddr.getHostName())
+        .setBindAddress(bindHost)
         .setPort(rpcAddr.getPort())
         .setPort(rpcAddr.getPort())
         .setNumHandlers(handlerCount)
         .setNumHandlers(handlerCount)
         .setVerbose(false)
         .setVerbose(false)
@@ -291,7 +308,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     }
 
 
     // The rpc-server port can be ephemeral... ensure we have the correct info
     // The rpc-server port can be ephemeral... ensure we have the correct info
-    clientRpcAddress = clientRpcServer.getListenerAddress();
+    InetSocketAddress listenAddr = clientRpcServer.getListenerAddress();
+      clientRpcAddress = new InetSocketAddress(
+          rpcAddr.getHostName(), listenAddr.getPort());
     nn.setRpcServerAddress(conf, clientRpcAddress);
     nn.setRpcServerAddress(conf, clientRpcAddress);
     
     
     minimumDataNodeVersion = conf.get(
     minimumDataNodeVersion = conf.get(
@@ -315,6 +334,12 @@ class NameNodeRpcServer implements NamenodeProtocols {
         NSQuotaExceededException.class,
         NSQuotaExceededException.class,
         DSQuotaExceededException.class);
         DSQuotaExceededException.class);
  }
  }
+
+  /** Allow access to the client RPC server for testing */
+  @VisibleForTesting
+  RPC.Server getClientRpcServer() {
+    return clientRpcServer;
+  }
   
   
   /**
   /**
    * Start client and service RPC servers.
    * Start client and service RPC servers.

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

@@ -32,6 +32,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
@@ -100,6 +101,20 @@ class NamenodeJspHelper {
     }
     }
   }
   }
 
 
+  static String getRollingUpgradeText(FSNamesystem fsn) {
+    DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
+    Map<String, Integer> list = dm.getDatanodesSoftwareVersions();
+    if(list.size() > 1) {
+      StringBuffer status = new StringBuffer("Rolling upgrades in progress. " +
+      "There are " + list.size() + " versions of datanodes currently live: ");
+      for(Map.Entry<String, Integer> ver: list.entrySet()) {
+        status.append(ver.getKey() + "(" + ver.getValue() + "), ");
+      }
+      return status.substring(0, status.length()-2);
+    }
+    return "";
+  }
+
   static String getInodeLimitText(FSNamesystem fsn) {
   static String getInodeLimitText(FSNamesystem fsn) {
     if (fsn == null) {
     if (fsn == null) {
       return "";
       return "";
@@ -803,7 +818,9 @@ class NamenodeJspHelper {
           + "<td align=\"right\" class=\"pcbpused\">"
           + "<td align=\"right\" class=\"pcbpused\">"
           + percentBpUsed
           + percentBpUsed
           + "<td align=\"right\" class=\"volfails\">"
           + "<td align=\"right\" class=\"volfails\">"
-          + d.getVolumeFailures() + "\n");
+          + d.getVolumeFailures()
+          + "<td align=\"right\" class=\"version\">"
+          + d.getSoftwareVersion() + "\n");
     }
     }
 
 
     void generateNodesList(ServletContext context, JspWriter out,
     void generateNodesList(ServletContext context, JspWriter out,
@@ -901,7 +918,9 @@ class NamenodeJspHelper {
                 + nodeHeaderStr("pcbpused")
                 + nodeHeaderStr("pcbpused")
                 + "> Block Pool<br>Used (%)" + " <th "
                 + "> Block Pool<br>Used (%)" + " <th "
                 + nodeHeaderStr("volfails")
                 + nodeHeaderStr("volfails")
-                +"> Failed Volumes\n");
+                +"> Failed Volumes <th "
+                + nodeHeaderStr("versionString")
+                +"> Version\n");
 
 
             JspHelper.sortNodeList(live, sorterField, sorterOrder);
             JspHelper.sortNodeList(live, sorterField, sorterOrder);
             for (int i = 0; i < live.size(); i++) {
             for (int i = 0; i < live.size(); i++) {

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

@@ -51,6 +51,18 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>dfs.namenode.rpc-bind-host</name>
+  <value></value>
+  <description>
+    The actual address the server will bind to. If this optional address is
+    set, the RPC server will bind to this address and the port specified in
+    dfs.namenode.rpc-address for the RPC server. It can also be specified
+    per name node or name service for HA/Federation. This is most useful for
+    making name node listen to all interfaces by setting to 0.0.0.0.
+  </description>
+</property>
+
 <property>
 <property>
   <name>dfs.namenode.servicerpc-address</name>
   <name>dfs.namenode.servicerpc-address</name>
   <value></value>
   <value></value>
@@ -64,6 +76,18 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>dfs.namenode.servicerpc-bind-host</name>
+  <value></value>
+  <description>
+    The actual address the server will bind to. If this optional address is
+    set, the service RPC server will bind to this address and the port 
+    specified in dfs.namenode.servicerpc-address. It can also be specified
+    per name node or name service for HA/Federation. This is most useful for
+    making name node listen to all interfaces by setting to 0.0.0.0.
+  </description>
+</property>
+
 <property>
 <property>
   <name>dfs.namenode.secondary.http-address</name>
   <name>dfs.namenode.secondary.http-address</name>
   <value>0.0.0.0:50090</value>
   <value>0.0.0.0:50090</value>

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp

@@ -65,6 +65,7 @@
 <h3>Cluster Summary</h3>
 <h3>Cluster Summary</h3>
 <b> <%= NamenodeJspHelper.getSecurityModeText()%> </b>
 <b> <%= NamenodeJspHelper.getSecurityModeText()%> </b>
 <b> <%= NamenodeJspHelper.getSafeModeText(fsn)%> </b>
 <b> <%= NamenodeJspHelper.getSafeModeText(fsn)%> </b>
+<b> <%= NamenodeJspHelper.getRollingUpgradeText(fsn)%> </b>
 <b> <%= NamenodeJspHelper.getInodeLimitText(fsn)%> </b>
 <b> <%= NamenodeJspHelper.getInodeLimitText(fsn)%> </b>
 <%= NamenodeJspHelper.getCorruptFilesWarning(fsn)%>
 <%= NamenodeJspHelper.getCorruptFilesWarning(fsn)%>
 
 

+ 148 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java

@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mortbay.log.Log;
+
+import static org.junit.Assert.*;
+
+public class TestDatanodeManager {
+
+  //The number of times the registration / removal of nodes should happen
+  final int NUM_ITERATIONS = 500;
+
+  /**
+   * This test sends a random sequence of node registrations and node removals
+   * to the DatanodeManager (of nodes with different IDs and versions), and
+   * checks that the DatanodeManager keeps a correct count of different software
+   * versions at all times.
+   */
+  @Test
+  public void testNumVersionsReportedCorrect() throws IOException {
+    //Create the DatanodeManager which will be tested
+    FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
+    Mockito.when(fsn.hasWriteLock()).thenReturn(true);
+    DatanodeManager dm = new DatanodeManager(Mockito.mock(BlockManager.class),
+      fsn, new Configuration());
+
+    //Seed the RNG with a known value so test failures are easier to reproduce
+    Random rng = new Random();
+    int seed = rng.nextInt();
+    rng = new Random(seed);
+    Log.info("Using seed " + seed + " for testing");
+
+    //A map of the Storage IDs to the DN registration it was registered with
+    HashMap <String, DatanodeRegistration> sIdToDnReg =
+      new HashMap<String, DatanodeRegistration>();
+
+    for(int i=0; i<NUM_ITERATIONS; ++i) {
+
+      //If true, remove a node for every 3rd time (if there's one)
+      if(rng.nextBoolean() && i%3 == 0 && sIdToDnReg.size()!=0) {
+        //Pick a random node.
+        int randomIndex = rng.nextInt() % sIdToDnReg.size();
+        //Iterate to that random position 
+        Iterator<Map.Entry<String, DatanodeRegistration>> it =
+          sIdToDnReg.entrySet().iterator();
+        for(int j=0; j<randomIndex-1; ++j) {
+          it.next();
+        }
+        DatanodeRegistration toRemove = it.next().getValue();
+        Log.info("Removing node " + toRemove.getStorageID() + " ip " +
+        toRemove.getXferAddr() + " version : " + toRemove.getSoftwareVersion());
+
+        //Remove that random node
+        dm.removeDatanode(toRemove);
+        it.remove();
+      }
+
+      // Otherwise register a node. This node may be a new / an old one
+      else {
+        //Pick a random storageID to register.
+        String storageID = "someStorageID" + rng.nextInt(5000);
+
+        DatanodeRegistration dr = Mockito.mock(DatanodeRegistration.class);
+        Mockito.when(dr.getStorageID()).thenReturn(storageID);
+
+        //If this storageID had already been registered before
+        if(sIdToDnReg.containsKey(storageID)) {
+          dr = sIdToDnReg.get(storageID);
+          //Half of the times, change the IP address
+          if(rng.nextBoolean()) {
+            dr.setIpAddr(dr.getIpAddr() + "newIP");
+          }
+        } else { //This storageID has never been registered
+          //Ensure IP address is unique to storageID
+          String ip = "someIP" + storageID;
+          Mockito.when(dr.getIpAddr()).thenReturn(ip);
+          Mockito.when(dr.getXferAddr()).thenReturn(ip + ":9000");
+          Mockito.when(dr.getXferPort()).thenReturn(9000);
+        }
+        //Pick a random version to register with
+        Mockito.when(dr.getSoftwareVersion()).thenReturn(
+          "version" + rng.nextInt(5));
+
+        Log.info("Registering node storageID: " + dr.getStorageID() +
+          ", version: " + dr.getSoftwareVersion() + ", IP address: "
+          + dr.getXferAddr());
+
+        //Register this random node
+        dm.registerDatanode(dr);
+        sIdToDnReg.put(storageID, dr);
+      }
+
+      //Verify DatanodeManager still has the right count
+      Map<String, Integer> mapToCheck = dm.getDatanodesSoftwareVersions();
+
+      //Remove counts from versions and make sure that after removing all nodes
+      //mapToCheck is empty
+      for(Entry<String, DatanodeRegistration> it: sIdToDnReg.entrySet()) {
+        String ver = it.getValue().getSoftwareVersion();
+        if(!mapToCheck.containsKey(ver)) {
+          throw new AssertionError("The correct number of datanodes of a "
+            + "version was not found on iteration " + i);
+        }
+        mapToCheck.put(ver, mapToCheck.get(ver) - 1);
+        if(mapToCheck.get(ver) == 0) {
+          mapToCheck.remove(ver);
+        }
+      }
+      for(Entry <String, Integer> entry: mapToCheck.entrySet()) {
+        Log.info("Still in map: " + entry.getKey() + " has "
+          + entry.getValue());
+      }
+      assertEquals("The map of version counts returned by DatanodeManager was"
+        + " not what it was expected to be on iteration " + i, 0,
+        mapToCheck.size());
+    }
+  }
+
+}

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java

@@ -0,0 +1,63 @@
+/**
+ * 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.
+ */
+
+/*
+ * Test the MiniDFSCluster functionality that allows "dfs.datanode.address",
+ * "dfs.datanode.http.address", and "dfs.datanode.ipc.address" to be
+ * configurable. The MiniDFSCluster.startDataNodes() API now has a parameter
+ * that will check these properties if told to do so.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.junit.Test;
+
+public class TestNameNodeRpcServer {
+
+  @Test
+  public void testNamenodeRpcBindAny() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+
+    // The name node in MiniDFSCluster only binds to 127.0.0.1.
+    // We can set the bind address to 0.0.0.0 to make it listen
+    // to all interfaces.
+    conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY, "0.0.0.0");
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      assertEquals("0.0.0.0", ((NameNodeRpcServer)cluster.getNameNodeRpc())
+          .getClientRpcServer().getListenerAddress().getHostName());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      // Reset the config
+      conf.unset(DFS_NAMENODE_RPC_BIND_HOST_KEY);
+    }
+  }
+}
+

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

@@ -243,6 +243,12 @@ Release 2.1.1-beta - UNRELEASED
     MAPREDUCE-5476. Changed MR AM recovery code to cleanup staging-directory
     MAPREDUCE-5476. Changed MR AM recovery code to cleanup staging-directory
     only after unregistering from the RM. (Jian He via vinodkv)
     only after unregistering from the RM. (Jian He via vinodkv)
 
 
+    MAPREDUCE-5483. revert MAPREDUCE-5357. (rkanter via tucu)
+
+    MAPREDUCE-5441. Changed MR AM to return RUNNING state if exiting when RM
+    commands to reboot, so that client can continue to track the overall job.
+    (Jian He via vinodkv)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -993,7 +993,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     }
     }
   }
   }
   
   
-  private static JobState getExternalState(JobStateInternal smState) {
+  private JobState getExternalState(JobStateInternal smState) {
     switch (smState) {
     switch (smState) {
     case KILL_WAIT:
     case KILL_WAIT:
     case KILL_ABORT:
     case KILL_ABORT:
@@ -1005,7 +1005,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     case FAIL_ABORT:
     case FAIL_ABORT:
       return JobState.FAILED;
       return JobState.FAILED;
     case REBOOT:
     case REBOOT:
-      return JobState.ERROR;
+      if (appContext.isLastAMRetry()) {
+        return JobState.ERROR;
+      } else {
+        // In case of not last retry, return the external state as RUNNING since
+        // otherwise JobClient will exit when it polls the AM for job state
+        return JobState.RUNNING;
+      }
     default:
     default:
       return JobState.valueOf(smState.name());
       return JobState.valueOf(smState.name());
     }
     }

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

@@ -29,6 +29,7 @@ import java.util.Iterator;
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
@@ -41,6 +42,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobUpdatedNodesEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobUpdatedNodesEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
@@ -51,12 +54,15 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.util.Clock;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
@@ -368,6 +374,47 @@ public class TestMRApp {
     app.waitForState(job, JobState.ERROR);
     app.waitForState(job, JobState.ERROR);
   }
   }
 
 
+  @Test
+  public void testJobRebootNotLastRetry() throws Exception {
+    MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
+    Job job = app.submit(new Configuration());
+    app.waitForState(job, JobState.RUNNING);
+    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    Iterator<Task> it = job.getTasks().values().iterator();
+    Task task = it.next();
+    app.waitForState(task, TaskState.RUNNING);
+
+    //send an reboot event
+    app.getContext().getEventHandler().handle(new JobEvent(job.getID(),
+      JobEventType.JOB_AM_REBOOT));
+
+    // return exteranl state as RUNNING since otherwise the JobClient will
+    // prematurely exit.
+    app.waitForState(job, JobState.RUNNING);
+  }
+
+  @Test
+  public void testJobRebootOnLastRetry() throws Exception {
+    // make startCount as 2 since this is last retry which equals to
+    // DEFAULT_MAX_AM_RETRY
+    MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true, 2);
+
+    Configuration conf = new Configuration();
+    Job job = app.submit(conf);
+    app.waitForState(job, JobState.RUNNING);
+    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    Iterator<Task> it = job.getTasks().values().iterator();
+    Task task = it.next();
+    app.waitForState(task, TaskState.RUNNING);
+
+    //send an reboot event
+    app.getContext().getEventHandler().handle(new JobEvent(job.getID(),
+      JobEventType.JOB_AM_REBOOT));
+
+    // return exteranl state as ERROR if this is the last retry
+    app.waitForState(job, JobState.ERROR);
+  }
+
   private final class MRAppWithSpiedJob extends MRApp {
   private final class MRAppWithSpiedJob extends MRApp {
     private JobImpl spiedJob;
     private JobImpl spiedJob;
 
 

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

@@ -142,7 +142,7 @@ public class TestJobImpl {
         "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ",
         "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ",
         "tag1,tag2");
         "tag1,tag2");
     dispatcher.register(EventType.class, jseHandler);
     dispatcher.register(EventType.class, jseHandler);
-    JobImpl job = createStubbedJob(conf, dispatcher, 0);
+    JobImpl job = createStubbedJob(conf, dispatcher, 0, null);
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
     job.handle(new JobStartEvent(job.getID()));
     job.handle(new JobStartEvent(job.getID()));
@@ -170,7 +170,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, null);
     completeJobTasks(job);
     completeJobTasks(job);
     assertJobState(job, JobStateInternal.COMMITTING);
     assertJobState(job, JobStateInternal.COMMITTING);
 
 
@@ -195,7 +195,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, null);
     completeJobTasks(job);
     completeJobTasks(job);
     assertJobState(job, JobStateInternal.COMMITTING);
     assertJobState(job, JobStateInternal.COMMITTING);
 
 
@@ -239,7 +239,9 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createStubbedJob(conf, dispatcher, 2);
+    AppContext mockContext = mock(AppContext.class);
+    when(mockContext.isLastAMRetry()).thenReturn(false);
+    JobImpl job = createStubbedJob(conf, dispatcher, 2, mockContext);
     JobId jobId = job.getID();
     JobId jobId = job.getID();
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
@@ -248,6 +250,10 @@ public class TestJobImpl {
 
 
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
     assertJobState(job, JobStateInternal.REBOOT);
     assertJobState(job, JobStateInternal.REBOOT);
+    // return the external state as RUNNING since otherwise JobClient will
+    // exit when it polls the AM for job state
+    Assert.assertEquals(JobState.RUNNING, job.getState());
+
     dispatcher.stop();
     dispatcher.stop();
     commitHandler.stop();
     commitHandler.stop();
   }
   }
@@ -256,6 +262,7 @@ public class TestJobImpl {
   public void testRebootedDuringCommit() throws Exception {
   public void testRebootedDuringCommit() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
+    conf.setInt(MRJobConfig.MR_AM_MAX_ATTEMPTS, 2);
     AsyncDispatcher dispatcher = new AsyncDispatcher();
     AsyncDispatcher dispatcher = new AsyncDispatcher();
     dispatcher.init(conf);
     dispatcher.init(conf);
     dispatcher.start();
     dispatcher.start();
@@ -266,13 +273,18 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
+    AppContext mockContext = mock(AppContext.class);
+    when(mockContext.isLastAMRetry()).thenReturn(true);
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, mockContext);
     completeJobTasks(job);
     completeJobTasks(job);
     assertJobState(job, JobStateInternal.COMMITTING);
     assertJobState(job, JobStateInternal.COMMITTING);
 
 
     syncBarrier.await();
     syncBarrier.await();
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
     assertJobState(job, JobStateInternal.REBOOT);
     assertJobState(job, JobStateInternal.REBOOT);
+    // return the external state as FAILED since this is last retry.
+    Assert.assertEquals(JobState.ERROR, job.getState());
+
     dispatcher.stop();
     dispatcher.stop();
     commitHandler.stop();
     commitHandler.stop();
   }
   }
@@ -301,7 +313,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createStubbedJob(conf, dispatcher, 2, null);
     JobId jobId = job.getID();
     JobId jobId = job.getID();
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
@@ -328,7 +340,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, null);
     completeJobTasks(job);
     completeJobTasks(job);
     assertJobState(job, JobStateInternal.COMMITTING);
     assertJobState(job, JobStateInternal.COMMITTING);
 
 
@@ -352,7 +364,7 @@ public class TestJobImpl {
         createCommitterEventHandler(dispatcher, committer);
         createCommitterEventHandler(dispatcher, committer);
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
-    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, null);
 
 
     //Fail one task. This should land the JobImpl in the FAIL_WAIT state
     //Fail one task. This should land the JobImpl in the FAIL_WAIT state
     job.handle(new JobTaskEvent(
     job.handle(new JobTaskEvent(
@@ -388,7 +400,7 @@ public class TestJobImpl {
     //Job has only 1 mapper task. No reducers
     //Job has only 1 mapper task. No reducers
     conf.setInt(MRJobConfig.NUM_REDUCES, 0);
     conf.setInt(MRJobConfig.NUM_REDUCES, 0);
     conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
     conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
-    JobImpl job = createRunningStubbedJob(conf, dispatcher, 1);
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 1, null);
 
 
     //Fail / finish all the tasks. This should land the JobImpl directly in the
     //Fail / finish all the tasks. This should land the JobImpl directly in the
     //FAIL_ABORT state
     //FAIL_ABORT state
@@ -440,7 +452,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createStubbedJob(conf, dispatcher, 2, null);
     JobId jobId = job.getID();
     JobId jobId = job.getID();
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
@@ -477,7 +489,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createStubbedJob(conf, dispatcher, 2, null);
     JobId jobId = job.getID();
     JobId jobId = job.getID();
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
@@ -687,7 +699,7 @@ public class TestJobImpl {
     commitHandler.init(conf);
     commitHandler.init(conf);
     commitHandler.start();
     commitHandler.start();
 
 
-    JobImpl job = createStubbedJob(conf, dispatcher, 2);
+    JobImpl job = createStubbedJob(conf, dispatcher, 2, null);
     JobId jobId = job.getID();
     JobId jobId = job.getID();
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
@@ -735,12 +747,12 @@ public class TestJobImpl {
   }
   }
 
 
   private static StubbedJob createStubbedJob(Configuration conf,
   private static StubbedJob createStubbedJob(Configuration conf,
-      Dispatcher dispatcher, int numSplits) {
+      Dispatcher dispatcher, int numSplits, AppContext appContext) {
     JobID jobID = JobID.forName("job_1234567890000_0001");
     JobID jobID = JobID.forName("job_1234567890000_0001");
     JobId jobId = TypeConverter.toYarn(jobID);
     JobId jobId = TypeConverter.toYarn(jobID);
     StubbedJob job = new StubbedJob(jobId,
     StubbedJob job = new StubbedJob(jobId,
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0),
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0),
-        conf,dispatcher.getEventHandler(), true, "somebody", numSplits);
+        conf,dispatcher.getEventHandler(), true, "somebody", numSplits, appContext);
     dispatcher.register(JobEventType.class, job);
     dispatcher.register(JobEventType.class, job);
     EventHandler mockHandler = mock(EventHandler.class);
     EventHandler mockHandler = mock(EventHandler.class);
     dispatcher.register(TaskEventType.class, mockHandler);
     dispatcher.register(TaskEventType.class, mockHandler);
@@ -751,8 +763,8 @@ public class TestJobImpl {
   }
   }
 
 
   private static StubbedJob createRunningStubbedJob(Configuration conf,
   private static StubbedJob createRunningStubbedJob(Configuration conf,
-      Dispatcher dispatcher, int numSplits) {
-    StubbedJob job = createStubbedJob(conf, dispatcher, numSplits);
+      Dispatcher dispatcher, int numSplits, AppContext appContext) {
+    StubbedJob job = createStubbedJob(conf, dispatcher, numSplits, appContext);
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
     job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
     assertJobState(job, JobStateInternal.INITED);
     assertJobState(job, JobStateInternal.INITED);
     job.handle(new JobStartEvent(job.getID()));
     job.handle(new JobStartEvent(job.getID()));
@@ -880,13 +892,13 @@ public class TestJobImpl {
     }
     }
 
 
     public StubbedJob(JobId jobId, ApplicationAttemptId applicationAttemptId,
     public StubbedJob(JobId jobId, ApplicationAttemptId applicationAttemptId,
-        Configuration conf, EventHandler eventHandler,
-        boolean newApiCommitter, String user, int numSplits) {
+        Configuration conf, EventHandler eventHandler, boolean newApiCommitter,
+        String user, int numSplits, AppContext appContext) {
       super(jobId, applicationAttemptId, conf, eventHandler,
       super(jobId, applicationAttemptId, conf, eventHandler,
           null, new JobTokenSecretManager(), new Credentials(),
           null, new JobTokenSecretManager(), new Credentials(),
           new SystemClock(), Collections.<TaskId, TaskInfo> emptyMap(),
           new SystemClock(), Collections.<TaskId, TaskInfo> emptyMap(),
           MRAppMetrics.create(), null, newApiCommitter, user,
           MRAppMetrics.create(), null, newApiCommitter, user,
-          System.currentTimeMillis(), null, null, null, null);
+          System.currentTimeMillis(), null, appContext, null, null);
 
 
       initTransition = getInitTransition(numSplits);
       initTransition = getInitTransition(numSplits);
       localFactory = stateMachineFactory.addTransition(JobStateInternal.NEW,
       localFactory = stateMachineFactory.addTransition(JobStateInternal.NEW,

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java

@@ -124,7 +124,6 @@ public class JobSubmissionFiles {
     } else {
     } else {
       fs.mkdirs(stagingArea, 
       fs.mkdirs(stagingArea, 
           new FsPermission(JOB_DIR_PERMISSION));
           new FsPermission(JOB_DIR_PERMISSION));
-      fs.setOwner(stagingArea, currentUser, null);
     }
     }
     return stagingArea;
     return stagingArea;
   }
   }

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

@@ -54,6 +54,18 @@ Release 2.1.1-beta - UNRELEASED
     YARN-942. In Fair Scheduler documentation, inconsistency on which
     YARN-942. In Fair Scheduler documentation, inconsistency on which
     properties have prefix (Akira Ajisaka via Sandy Ryza)
     properties have prefix (Akira Ajisaka via Sandy Ryza)
 
 
+    YARN-1083. Changed ResourceManager to fail when the expiry interval is less
+    than the configured node-heartbeat interval. (Zhijie Shen via vinodkv)
+
+    YARN-1081. Made a trivial change to YARN node CLI header to avoid potential
+    confusion. (Akira AJISAKA via vinodkv)
+
+    YARN-1034. Remove "experimental" in the Fair Scheduler documentation.
+    (Karthik Kambatla via Sandy Ryza)
+
+    YARN-1080. Improved help message for "yarn logs" command. (Xuan Gong via
+    vinodkv)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -105,6 +117,15 @@ Release 2.1.1-beta - UNRELEASED
     YARN-1008. MiniYARNCluster with multiple nodemanagers, all nodes have same 
     YARN-1008. MiniYARNCluster with multiple nodemanagers, all nodes have same 
     key for allocations. (tucu)
     key for allocations. (tucu)
 
 
+    YARN-981. Fixed YARN webapp so that /logs servlet works like before. (Jian He
+    via vinodkv)
+
+    YARN-602. Fixed NodeManager to not let users override some mandatory 
+    environmental variables. (Kenji Kikushima  via vinodkv)
+
+    YARN-1101. Active nodes can be decremented below 0 (Robert Parker 
+    via tgraves)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -1223,6 +1244,9 @@ Release 0.23.10 - UNRELEASED
 
 
     YARN-337. RM handles killed application tracking URL poorly (jlowe)
     YARN-337. RM handles killed application tracking URL poorly (jlowe)
 
 
+    YARN-1101. Active nodes can be decremented below 0 (Robert Parker 
+    via tgraves)
+
 Release 0.23.9 - 2013-07-08
 Release 0.23.9 - 2013-07-08
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 @Private
 @Private
 @Unstable
 @Unstable
 public class NodeCLI extends YarnCLI {
 public class NodeCLI extends YarnCLI {
-  private static final String NODES_PATTERN = "%16s\t%15s\t%17s\t%18s" +
+  private static final String NODES_PATTERN = "%16s\t%15s\t%17s\t%28s" +
     System.getProperty("line.separator");
     System.getProperty("line.separator");
 
 
   private static final String NODE_STATE_CMD = "states";
   private static final String NODE_STATE_CMD = "states";
@@ -133,7 +133,7 @@ public class NodeCLI extends YarnCLI {
                                        nodeStates.toArray(new NodeState[0]));
                                        nodeStates.toArray(new NodeState[0]));
     writer.println("Total Nodes:" + nodesReport.size());
     writer.println("Total Nodes:" + nodesReport.size());
     writer.printf(NODES_PATTERN, "Node-Id", "Node-State", "Node-Http-Address",
     writer.printf(NODES_PATTERN, "Node-Id", "Node-State", "Node-Http-Address",
-        "Running-Containers");
+        "Number-of-Running-Containers");
     for (NodeReport nodeReport : nodesReport) {
     for (NodeReport nodeReport : nodesReport) {
       writer.printf(NODES_PATTERN, nodeReport.getNodeId(), nodeReport
       writer.printf(NODES_PATTERN, nodeReport.getNodeId(), nodeReport
           .getNodeState(), nodeReport.getHttpAddress(), nodeReport
           .getNodeState(), nodeReport.getHttpAddress(), nodeReport

+ 46 - 46
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java

@@ -543,9 +543,9 @@ public class TestYarnCLI {
     PrintWriter pw = new PrintWriter(baos);
     PrintWriter pw = new PrintWriter(baos);
     pw.println("Total Nodes:1");
     pw.println("Total Nodes:1");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t            NEW\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t            NEW\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     String nodesReportStr = baos.toString("UTF-8");
     String nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -564,11 +564,11 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:2");
     pw.println("Total Nodes:2");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t        RUNNING\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host1:0\t        RUNNING\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t        RUNNING\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host1:0\t        RUNNING\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -593,9 +593,9 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:1");
     pw.println("Total Nodes:1");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t      UNHEALTHY\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t      UNHEALTHY\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -614,9 +614,9 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:1");
     pw.println("Total Nodes:1");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t DECOMMISSIONED\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t DECOMMISSIONED\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -635,9 +635,9 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:1");
     pw.println("Total Nodes:1");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t       REBOOTED\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t       REBOOTED\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -656,9 +656,9 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:1");
     pw.println("Total Nodes:1");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t           LOST\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t           LOST\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -681,17 +681,17 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:5");
     pw.println("Total Nodes:5");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t            NEW\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t        RUNNING\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host1:0\t        RUNNING\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t       REBOOTED\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t           LOST\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t            NEW\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t        RUNNING\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host1:0\t        RUNNING\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t       REBOOTED\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t           LOST\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
@@ -712,21 +712,21 @@ public class TestYarnCLI {
     pw = new PrintWriter(baos);
     pw = new PrintWriter(baos);
     pw.println("Total Nodes:7");
     pw.println("Total Nodes:7");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
     pw.print("         Node-Id\t     Node-State\tNode-Http-Address\t");
-    pw.println("Running-Containers");
-    pw.print("         host0:0\t            NEW\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t        RUNNING\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host1:0\t        RUNNING\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t      UNHEALTHY\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t DECOMMISSIONED\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t       REBOOTED\t       host1:8888");
-    pw.println("\t                 0");
-    pw.print("         host0:0\t           LOST\t       host1:8888");
-    pw.println("\t                 0");
+    pw.println("Number-of-Running-Containers");
+    pw.print("         host0:0\t            NEW\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t        RUNNING\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host1:0\t        RUNNING\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t      UNHEALTHY\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t DECOMMISSIONED\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t       REBOOTED\t       host1:8888\t");
+    pw.println("                           0");
+    pw.print("         host0:0\t           LOST\t       host1:8888\t");
+    pw.println("                           0");
     pw.close();
     pw.close();
     nodesReportStr = baos.toString("UTF-8");
     nodesReportStr = baos.toString("UTF-8");
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     Assert.assertEquals(nodesReportStr, sysOutStream.toString());

+ 21 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java

@@ -72,10 +72,18 @@ public class LogDumper extends Configured implements Tool {
       + "nodename:port (must be specified if container id is specified)");
       + "nodename:port (must be specified if container id is specified)");
     opts.addOption(APP_OWNER_OPTION, true,
     opts.addOption(APP_OWNER_OPTION, true,
       "AppOwner (assumed to be current user if not specified)");
       "AppOwner (assumed to be current user if not specified)");
+    opts.getOption(APPLICATION_ID_OPTION).setArgName("Application ID");
+    opts.getOption(CONTAINER_ID_OPTION).setArgName("Container ID");
+    opts.getOption(NODE_ADDRESS_OPTION).setArgName("Node Address");
+    opts.getOption(APP_OWNER_OPTION).setArgName("Application Owner");
+
+    Options printOpts = new Options();
+    printOpts.addOption(opts.getOption(CONTAINER_ID_OPTION));
+    printOpts.addOption(opts.getOption(NODE_ADDRESS_OPTION));
+    printOpts.addOption(opts.getOption(APP_OWNER_OPTION));
 
 
     if (args.length < 1) {
     if (args.length < 1) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("general options are: ", opts);
+      printHelpMessage(printOpts);
       return -1;
       return -1;
     }
     }
 
 
@@ -92,16 +100,13 @@ public class LogDumper extends Configured implements Tool {
       appOwner = commandLine.getOptionValue(APP_OWNER_OPTION);
       appOwner = commandLine.getOptionValue(APP_OWNER_OPTION);
     } catch (ParseException e) {
     } catch (ParseException e) {
       System.out.println("options parsing failed: " + e.getMessage());
       System.out.println("options parsing failed: " + e.getMessage());
-
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("general options are: ", opts);
+      printHelpMessage(printOpts);
       return -1;
       return -1;
     }
     }
 
 
     if (appIdStr == null) {
     if (appIdStr == null) {
       System.out.println("ApplicationId cannot be null!");
       System.out.println("ApplicationId cannot be null!");
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("general options are: ", opts);
+      printHelpMessage(printOpts);
       return -1;
       return -1;
     }
     }
 
 
@@ -119,8 +124,7 @@ public class LogDumper extends Configured implements Tool {
     } else if ((containerIdStr == null && nodeAddress != null)
     } else if ((containerIdStr == null && nodeAddress != null)
         || (containerIdStr != null && nodeAddress == null)) {
         || (containerIdStr != null && nodeAddress == null)) {
       System.out.println("ContainerId or NodeAddress cannot be null!");
       System.out.println("ContainerId or NodeAddress cannot be null!");
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("general options are: ", opts);
+      printHelpMessage(printOpts);
       resultCode = -1;
       resultCode = -1;
     } else {
     } else {
       Path remoteRootLogDir =
       Path remoteRootLogDir =
@@ -255,4 +259,12 @@ public class LogDumper extends Configured implements Tool {
     int exitCode = logDumper.run(args);
     int exitCode = logDumper.run(args);
     System.exit(exitCode);
     System.exit(exitCode);
   }
   }
+
+  private void printHelpMessage(Options options) {
+    System.out.println("Retrieve logs for completed YARN applications.");
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("yarn logs -applicationId <application ID> [OPTIONS]", new Options());
+    formatter.setSyntaxPrefix("");
+    formatter.printHelp("general options are:", options);
+  }
 }
 }

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

@@ -242,7 +242,10 @@ public class WebApps {
         for(Map.Entry<String, Object> entry : attributes.entrySet()) {
         for(Map.Entry<String, Object> entry : attributes.entrySet()) {
           server.setAttribute(entry.getKey(), entry.getValue());
           server.setAttribute(entry.getKey(), entry.getValue());
         }
         }
-        server.addGlobalFilter("guice", GuiceFilter.class.getName(), null);
+        String webAppPath = "/" + name + "/*";
+        server.defineFilter(server.getWebAppContext(), "guice",
+          GuiceFilter.class.getName(), null, new String[] { webAppPath, "/" });
+
         webapp.setConf(conf);
         webapp.setConf(conf);
         webapp.setHttpServer(server);
         webapp.setHttpServer(server);
         server.start();
         server.start();

+ 42 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestLogDumper.java

@@ -19,14 +19,30 @@
 package org.apache.hadoop.yarn.logaggregation;
 package org.apache.hadoop.yarn.logaggregation;
 
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+
+import junit.framework.Assert;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestLogDumper {
 public class TestLogDumper {
+  ByteArrayOutputStream sysOutStream;
+  private PrintStream sysOut;
+
+  @Before
+  public void setUp() {
+    sysOutStream = new ByteArrayOutputStream();
+    sysOut =  new PrintStream(sysOutStream);
+    System.setOut(sysOut);
+  }
+
   @Test
   @Test
   public void testFailResultCodes() throws Exception {
   public void testFailResultCodes() throws Exception {
     Configuration conf = new YarnConfiguration();
     Configuration conf = new YarnConfiguration();
@@ -44,4 +60,30 @@ public class TestLogDumper {
         "nonexistentnode:1234", "nobody");
         "nonexistentnode:1234", "nobody");
     assertTrue("Should return an error code", exitCode != 0);
     assertTrue("Should return an error code", exitCode != 0);
   }
   }
+
+  @Test
+  public void testHelpMessage() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    LogDumper dumper = new LogDumper();
+    dumper.setConf(conf);
+
+    int exitCode = dumper.run(new String[]{});
+    assertTrue(exitCode == -1);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Retrieve logs for completed YARN applications.");
+    pw.println("usage: yarn logs -applicationId <application ID> [OPTIONS]");
+    pw.println();
+    pw.println("general options are:");
+    pw.println(" -appOwner <Application Owner>   AppOwner (assumed to be current user if");
+    pw.println("                                 not specified)");
+    pw.println(" -containerId <Container ID>     ContainerId (must be specified if node");
+    pw.println("                                 address is specified)");
+    pw.println(" -nodeAddress <Node Address>     NodeAddress in the format nodename:port");
+    pw.println("                                 (must be specified if container id is");
+    pw.println("                                 specified)");
+    pw.close();
+    String appReportStr = baos.toString("UTF-8");
+    Assert.assertEquals(appReportStr, sysOutStream.toString());
+  }
 }
 }

+ 5 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -588,20 +588,18 @@ public class ContainerLaunch implements Callable<Integer> {
     environment.put(Environment.LOG_DIRS.name(),
     environment.put(Environment.LOG_DIRS.name(),
       StringUtils.join(",", containerLogDirs));
       StringUtils.join(",", containerLogDirs));
 
 
-    putEnvIfNotNull(environment, Environment.USER.name(), container.getUser());
+    environment.put(Environment.USER.name(), container.getUser());
     
     
-    putEnvIfNotNull(environment, 
-        Environment.LOGNAME.name(),container.getUser());
-    
-    putEnvIfNotNull(environment, 
-        Environment.HOME.name(),
+    environment.put(Environment.LOGNAME.name(), container.getUser());
+
+    environment.put(Environment.HOME.name(),
         conf.get(
         conf.get(
             YarnConfiguration.NM_USER_HOME_DIR, 
             YarnConfiguration.NM_USER_HOME_DIR, 
             YarnConfiguration.DEFAULT_NM_USER_HOME_DIR
             YarnConfiguration.DEFAULT_NM_USER_HOME_DIR
             )
             )
         );
         );
     
     
-    putEnvIfNotNull(environment, Environment.PWD.name(), pwd.toString());
+    environment.put(Environment.PWD.name(), pwd.toString());
     
     
     putEnvIfNotNull(environment, 
     putEnvIfNotNull(environment, 
         Environment.HADOOP_CONF_DIR.name(), 
         Environment.HADOOP_CONF_DIR.name(), 

+ 57 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -346,7 +346,6 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ApplicationAttemptId appAttemptId =
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
         ApplicationAttemptId.newInstance(appId, 1);
 
 
-    int port = 12345;
     ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
     ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
     Map<String, String> userSetEnv = new HashMap<String, String>();
     Map<String, String> userSetEnv = new HashMap<String, String>();
     userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
     userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
@@ -354,6 +353,11 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     userSetEnv.put(Environment.NM_PORT.name(), "user_set_NM_PORT");
     userSetEnv.put(Environment.NM_PORT.name(), "user_set_NM_PORT");
     userSetEnv.put(Environment.NM_HTTP_PORT.name(), "user_set_NM_HTTP_PORT");
     userSetEnv.put(Environment.NM_HTTP_PORT.name(), "user_set_NM_HTTP_PORT");
     userSetEnv.put(Environment.LOCAL_DIRS.name(), "user_set_LOCAL_DIR");
     userSetEnv.put(Environment.LOCAL_DIRS.name(), "user_set_LOCAL_DIR");
+    userSetEnv.put(Environment.USER.key(), "user_set_" +
+    	Environment.USER.key());
+    userSetEnv.put(Environment.LOGNAME.name(), "user_set_LOGNAME");
+    userSetEnv.put(Environment.PWD.name(), "user_set_PWD");
+    userSetEnv.put(Environment.HOME.name(), "user_set_HOME");
     containerLaunchContext.setEnvironment(userSetEnv);
     containerLaunchContext.setEnvironment(userSetEnv);
 
 
     File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
     File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
@@ -371,6 +375,14 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
           + processStartFile);
           + processStartFile);
       fileWriter.println("@echo " + Environment.LOCAL_DIRS.$() + ">> "
       fileWriter.println("@echo " + Environment.LOCAL_DIRS.$() + ">> "
           + processStartFile);
           + processStartFile);
+      fileWriter.println("@echo " + Environment.USER.$() + ">> "
+    	  + processStartFile);
+      fileWriter.println("@echo " + Environment.LOGNAME.$() + ">> "
+          + processStartFile);
+      fileWriter.println("@echo " + Environment.PWD.$() + ">> "
+    	  + processStartFile);
+      fileWriter.println("@echo " + Environment.HOME.$() + ">> "
+          + processStartFile);
       fileWriter.println("@echo " + cId + ">> " + processStartFile);
       fileWriter.println("@echo " + cId + ">> " + processStartFile);
       fileWriter.println("@ping -n 100 127.0.0.1 >nul");
       fileWriter.println("@ping -n 100 127.0.0.1 >nul");
     } else {
     } else {
@@ -385,6 +397,14 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
           + processStartFile);
           + processStartFile);
       fileWriter.write("\necho $" + Environment.LOCAL_DIRS.name() + " >> "
       fileWriter.write("\necho $" + Environment.LOCAL_DIRS.name() + " >> "
           + processStartFile);
           + processStartFile);
+      fileWriter.write("\necho $" + Environment.USER.name() + " >> "
+          + processStartFile);
+      fileWriter.write("\necho $" + Environment.LOGNAME.name() + " >> "
+          + processStartFile);
+      fileWriter.write("\necho $" + Environment.PWD.name() + " >> "
+          + processStartFile);
+      fileWriter.write("\necho $" + Environment.HOME.name() + " >> "
+          + processStartFile);
       fileWriter.write("\necho $$ >> " + processStartFile);
       fileWriter.write("\necho $$ >> " + processStartFile);
       fileWriter.write("\nexec sleep 100");
       fileWriter.write("\nexec sleep 100");
     }
     }
@@ -452,6 +472,22 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       reader.readLine());
       reader.readLine());
     Assert.assertEquals(String.valueOf(HTTP_PORT), reader.readLine());
     Assert.assertEquals(String.valueOf(HTTP_PORT), reader.readLine());
     Assert.assertEquals(StringUtils.join(",", appDirs), reader.readLine());
     Assert.assertEquals(StringUtils.join(",", appDirs), reader.readLine());
+    Assert.assertEquals(user, reader.readLine());
+    Assert.assertEquals(user, reader.readLine());
+    String obtainedPWD = reader.readLine();
+    boolean found = false;
+    for (Path localDir : appDirs) {
+      if (new Path(localDir, cId.toString()).toString().equals(obtainedPWD)) {
+        found = true;
+        break;
+      }
+    }
+    Assert.assertTrue("Wrong local-dir found : " + obtainedPWD, found);
+    Assert.assertEquals(
+        conf.get(
+              YarnConfiguration.NM_USER_HOME_DIR, 
+              YarnConfiguration.DEFAULT_NM_USER_HOME_DIR),
+        reader.readLine());
 
 
     Assert.assertEquals(cId.toString(), containerLaunchContext
     Assert.assertEquals(cId.toString(), containerLaunchContext
         .getEnvironment().get(Environment.CONTAINER_ID.name()));
         .getEnvironment().get(Environment.CONTAINER_ID.name()));
@@ -465,6 +501,26 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         .getEnvironment().get(Environment.LOCAL_DIRS.name()));
         .getEnvironment().get(Environment.LOCAL_DIRS.name()));
     Assert.assertEquals(StringUtils.join(",", containerLogDirs),
     Assert.assertEquals(StringUtils.join(",", containerLogDirs),
       containerLaunchContext.getEnvironment().get(Environment.LOG_DIRS.name()));
       containerLaunchContext.getEnvironment().get(Environment.LOG_DIRS.name()));
+    Assert.assertEquals(user, containerLaunchContext.getEnvironment()
+    	.get(Environment.USER.name()));
+    Assert.assertEquals(user, containerLaunchContext.getEnvironment()
+    	.get(Environment.LOGNAME.name()));
+    found = false;
+    obtainedPWD =
+        containerLaunchContext.getEnvironment().get(Environment.PWD.name());
+    for (Path localDir : appDirs) {
+      if (new Path(localDir, cId.toString()).toString().equals(obtainedPWD)) {
+        found = true;
+        break;
+      }
+    }
+    Assert.assertTrue("Wrong local-dir found : " + obtainedPWD, found);
+    Assert.assertEquals(
+        conf.get(
+    	        YarnConfiguration.NM_USER_HOME_DIR, 
+    	        YarnConfiguration.DEFAULT_NM_USER_HOME_DIR),
+    	containerLaunchContext.getEnvironment()
+    		.get(Environment.HOME.name()));
 
 
     // Get the pid of the process
     // Get the pid of the process
     String pid = reader.readLine().trim();
     String pid = reader.readLine().trim();
@@ -538,7 +594,6 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
 
     ContainerLaunchContext containerLaunchContext = 
     ContainerLaunchContext containerLaunchContext = 
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    int port = 12345;
 
 
     // upload the script file so that the container can run it
     // upload the script file so that the container can run it
     URL resource_alpha =
     URL resource_alpha =

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

@@ -365,6 +365,20 @@ public class ResourceManager extends CompositeService implements Recoverable {
           + ", " + YarnConfiguration.RM_AM_MAX_ATTEMPTS
           + ", " + YarnConfiguration.RM_AM_MAX_ATTEMPTS
           + "=" + globalMaxAppAttempts + ", it should be a positive integer.");
           + "=" + globalMaxAppAttempts + ", it should be a positive integer.");
     }
     }
+
+    // validate expireIntvl >= heartbeatIntvl
+    long expireIntvl = conf.getLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
+    long heartbeatIntvl =
+        conf.getLong(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS,
+            YarnConfiguration.DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS);
+    if (expireIntvl < heartbeatIntvl) {
+      throw new YarnRuntimeException("Nodemanager expiry interval should be no"
+          + " less than heartbeat interval, "
+          + YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS + "=" + expireIntvl
+          + ", " + YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS + "="
+          + heartbeatIntvl);
+    }
   }
   }
 
 
   @Private
   @Private

+ 16 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -393,9 +393,18 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     }
     }
   }
   }
 
 
-  private void updateMetricsForDeactivatedNode(NodeState finalState) {
+  private void updateMetricsForDeactivatedNode(NodeState initialState,
+                                               NodeState finalState) {
     ClusterMetrics metrics = ClusterMetrics.getMetrics();
     ClusterMetrics metrics = ClusterMetrics.getMetrics();
-    metrics.decrNumActiveNodes();
+
+    switch (initialState) {
+      case RUNNING:
+        metrics.decrNumActiveNodes();
+        break;
+      case UNHEALTHY:
+        metrics.decrNumUnhealthyNMs();
+        break;
+    }
 
 
     switch (finalState) {
     switch (finalState) {
     case DECOMMISSIONED:
     case DECOMMISSIONED:
@@ -505,7 +514,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       // If the current state is NodeState.UNHEALTHY
       // If the current state is NodeState.UNHEALTHY
       // Then node is already been removed from the
       // Then node is already been removed from the
       // Scheduler
       // Scheduler
-      if (!rmNode.getState().equals(NodeState.UNHEALTHY)) {
+      NodeState initialState = rmNode.getState();
+      if (!initialState.equals(NodeState.UNHEALTHY)) {
         rmNode.context.getDispatcher().getEventHandler()
         rmNode.context.getDispatcher().getEventHandler()
           .handle(new NodeRemovedSchedulerEvent(rmNode));
           .handle(new NodeRemovedSchedulerEvent(rmNode));
       }
       }
@@ -520,7 +530,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       rmNode.context.getInactiveRMNodes().put(rmNode.nodeId.getHost(), rmNode);
       rmNode.context.getInactiveRMNodes().put(rmNode.nodeId.getHost(), rmNode);
 
 
       //Update the metrics
       //Update the metrics
-      rmNode.updateMetricsForDeactivatedNode(finalState);
+      rmNode.updateMetricsForDeactivatedNode(initialState, finalState);
     }
     }
   }
   }
 
 
@@ -550,7 +560,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
             new NodesListManagerEvent(
             new NodesListManagerEvent(
                 NodesListManagerEventType.NODE_UNUSABLE, rmNode));
                 NodesListManagerEventType.NODE_UNUSABLE, rmNode));
         // Update metrics
         // Update metrics
-        rmNode.updateMetricsForDeactivatedNode(NodeState.UNHEALTHY);
+        rmNode.updateMetricsForDeactivatedNode(rmNode.getState(),
+            NodeState.UNHEALTHY);
         return NodeState.UNHEALTHY;
         return NodeState.UNHEALTHY;
       }
       }
 
 

+ 42 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java

@@ -267,7 +267,21 @@ public class TestRMNodeTransitions {
   @Test
   @Test
   public void testUnhealthyExpire() {
   public void testUnhealthyExpire() {
     RMNodeImpl node = getUnhealthyNode();
     RMNodeImpl node = getUnhealthyNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
     node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE));
     node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE));
+    Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost + 1, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy - 1, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes",
+        initialDecommissioned, cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted, cm.getNumRebootedNMs());
     Assert.assertEquals(NodeState.LOST, node.getState());
     Assert.assertEquals(NodeState.LOST, node.getState());
   }
   }
   
   
@@ -291,8 +305,22 @@ public class TestRMNodeTransitions {
   @Test
   @Test
   public void testUnhealthyDecommission() {
   public void testUnhealthyDecommission() {
     RMNodeImpl node = getUnhealthyNode();
     RMNodeImpl node = getUnhealthyNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
     node.handle(new RMNodeEvent(node.getNodeID(),
     node.handle(new RMNodeEvent(node.getNodeID(),
         RMNodeEventType.DECOMMISSION));
         RMNodeEventType.DECOMMISSION));
+    Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy - 1, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes",
+        initialDecommissioned + 1, cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted, cm.getNumRebootedNMs());
     Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
     Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState());
   }
   }
 
 
@@ -307,8 +335,22 @@ public class TestRMNodeTransitions {
   @Test
   @Test
   public void testUnhealthyRebooting() {
   public void testUnhealthyRebooting() {
     RMNodeImpl node = getUnhealthyNode();
     RMNodeImpl node = getUnhealthyNode();
+    ClusterMetrics cm = ClusterMetrics.getMetrics();
+    int initialActive = cm.getNumActiveNMs();
+    int initialLost = cm.getNumLostNMs();
+    int initialUnhealthy = cm.getUnhealthyNMs();
+    int initialDecommissioned = cm.getNumDecommisionedNMs();
+    int initialRebooted = cm.getNumRebootedNMs();
     node.handle(new RMNodeEvent(node.getNodeID(),
     node.handle(new RMNodeEvent(node.getNodeID(),
         RMNodeEventType.REBOOTING));
         RMNodeEventType.REBOOTING));
+    Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs());
+    Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
+    Assert.assertEquals("Unhealthy Nodes",
+        initialUnhealthy - 1, cm.getUnhealthyNMs());
+    Assert.assertEquals("Decommissioned Nodes",
+        initialDecommissioned, cm.getNumDecommisionedNMs());
+    Assert.assertEquals("Rebooted Nodes",
+        initialRebooted + 1, cm.getNumRebootedNMs());
     Assert.assertEquals(NodeState.REBOOTED, node.getState());
     Assert.assertEquals(NodeState.REBOOTED, node.getState());
   }
   }
 
 

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java

@@ -203,4 +203,21 @@ public class TestResourceManager {
     }
     }
   }
   }
 
 
+  @Test
+  public void testNMExpiryAndHeartbeatIntervalsValidation() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 1000);
+    conf.setLong(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 1001);
+    resourceManager = new ResourceManager();;
+    try {
+      resourceManager.init(conf);
+    } catch (YarnRuntimeException e) {
+      // Exception is expected.
+      if (!e.getMessage().startsWith("Nodemanager expiry interval should be no"
+          + " less than heartbeat interval")) {
+        throw e;
+      }
+    }
+  }
+
 }
 }

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -25,8 +25,7 @@ Hadoop MapReduce Next Generation - Fair Scheduler
 * {Purpose} 
 * {Purpose} 
 
 
   This document describes the <<<FairScheduler>>>, a pluggable scheduler for Hadoop 
   This document describes the <<<FairScheduler>>>, a pluggable scheduler for Hadoop 
-  which provides a way to share large clusters. <<NOTE:>> The Fair Scheduler 
-  implementation is currently under development and should be considered experimental.
+  that allows YARN applications to share resources in large clusters fairly.
 
 
 * {Introduction}
 * {Introduction}