浏览代码

Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1569833 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父节点
当前提交
909eb7e4f1
共有 27 个文件被更改,包括 301 次插入127 次删除
  1. 14 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 10 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java
  3. 28 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  4. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
  5. 4 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  7. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
  10. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  12. 16 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
  13. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
  14. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
  15. 12 0
      hadoop-mapreduce-project/CHANGES.txt
  16. 15 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
  17. 26 0
      hadoop-yarn-project/CHANGES.txt
  18. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  19. 16 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  20. 10 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java
  21. 21 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  22. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  23. 28 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  24. 8 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java
  25. 21 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  26. 21 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  27. 15 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java

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

@@ -301,6 +301,18 @@ Trunk (Unreleased)
 
     HADOOP-8589. ViewFs tests fail when tests and home dirs are nested (sanjay Radia)
 
+Release 2.5.0 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES 
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -345,6 +357,8 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10249. LdapGroupsMapping should trim ldap password read from file.
     (Dilli Armugam via suresh)
 
+    HADOOP-10346. Deadlock while logging tokens (jlowe)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 10 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java

@@ -105,18 +105,21 @@ public class Token<T extends TokenIdentifier> implements Writable {
     return identifier;
   }
   
-  private static synchronized Class<? extends TokenIdentifier>
+  private static Class<? extends TokenIdentifier>
       getClassForIdentifier(Text kind) {
-    if (tokenKindMap == null) {
-      tokenKindMap = Maps.newHashMap();
-      for (TokenIdentifier id : ServiceLoader.load(TokenIdentifier.class)) {
-        tokenKindMap.put(id.getKind(), id.getClass());
+    Class<? extends TokenIdentifier> cls = null;
+    synchronized (Token.class) {
+      if (tokenKindMap == null) {
+        tokenKindMap = Maps.newHashMap();
+        for (TokenIdentifier id : ServiceLoader.load(TokenIdentifier.class)) {
+          tokenKindMap.put(id.getKind(), id.getClass());
+        }
       }
+      cls = tokenKindMap.get(kind);
     }
-    Class<? extends TokenIdentifier> cls = tokenKindMap.get(kind);
     if (cls == null) {
       LOG.warn("Cannot find class for token kind " + kind);
-       return null;
+      return null;
     }
     return cls;
   }

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

@@ -259,6 +259,18 @@ Trunk (Unreleased)
     HDFS-5794. Fix the inconsistency of layout version number of 
     ADD_DATANODE_AND_STORAGE_UUIDS between trunk and branch-2. (jing9)
 
+Release 2.5.0 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES 
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -411,13 +423,25 @@ Release 2.4.0 - UNRELEASED
     HDFS-5716. Allow WebHDFS to use pluggable authentication filter
     (Haohui Mai via brandonli)
 
-    HDFS-5953. TestBlockReaderFactory fails in trunk. (Akira Ajisaka via wang)
+    HDFS-5953. TestBlockReaderFactory fails if libhadoop.so has not been built.
+    (Akira Ajisaka via wang)
 
     HDFS-5759. Web UI does not show up during the period of loading FSImage.
     (Haohui Mai via Arpit Agarwal)
 
     HDFS-5942. Fix javadoc in OfflineImageViewer. (Akira Ajisaka via cnauroth)
 
+    HDFS-5780. TestRBWBlockInvalidation times out intemittently. (Mit Desai
+    via kihwal)
+
+    HDFS-5803. TestBalancer.testBalancer0 fails. (Chen He via kihwal)
+
+    HDFS-5893. HftpFileSystem.RangeHeaderUrlOpener uses the default
+    URLConnectionFactory which does not import SSL certificates. (Haohui Mai via
+    jing9)
+
+    HDFS-5961. OIV cannot load fsimages containing a symbolic link. (kihwal)
+
   BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
 
     HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)
@@ -472,6 +496,9 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5847. Consolidate INodeReference into a separate section. (jing9)
 
+    HDFS-5959. Fix typo at section name in FSImageFormatProtobuf.java.
+    (Akira Ajisaka via suresh)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -237,7 +237,7 @@ public final class FSImageFormatProtobuf {
           inodeLoader.loadINodeSection(in);
         }
           break;
-        case INODE_REFRENCE:
+        case INODE_REFERENCE:
           snapshotLoader.loadINodeReferenceSection(in);
           break;
         case INODE_DIR:
@@ -551,7 +551,7 @@ public final class FSImageFormatProtobuf {
     STRING_TABLE("STRING_TABLE"),
     EXTENDED_ACL("EXTENDED_ACL"),
     INODE("INODE"),
-    INODE_REFRENCE("INODE_REFRENCE"),
+    INODE_REFERENCE("INODE_REFERENCE"),
     SNAPSHOT("SNAPSHOT"),
     INODE_DIR("INODE_DIR"),
     FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"),

+ 4 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -27,7 +27,6 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -61,18 +60,13 @@ public class FileDataServlet extends DfsServlet {
     } else {
       hostname = host.getIpAddr();
     }
-    int port = host.getInfoPort();
-    if ("https".equals(scheme)) {
-      final Integer portObject = (Integer) getServletContext().getAttribute(
-          DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY);
-      if (portObject != null) {
-        port = portObject;
-      }
-    }
+
+    int port = "https".equals(scheme) ? host.getInfoSecurePort() : host
+        .getInfoPort();
 
     String dtParam = "";
     if (dt != null) {
-      dtParam=JspHelper.getDelegationTokenUrlParam(dt);
+      dtParam = JspHelper.getDelegationTokenUrlParam(dt);
     }
 
     // Add namenode address to the url params

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -402,7 +402,7 @@ public class FSImageFormatPBSnapshot {
         INodeReferenceSection.INodeReference.Builder rb = buildINodeReference(ref);
         rb.build().writeDelimitedTo(out);
       }
-      parent.commitSection(headers, SectionName.INODE_REFRENCE);
+      parent.commitSection(headers, SectionName.INODE_REFERENCE);
     }
 
     private INodeReferenceSection.INodeReference.Builder buildINodeReference(

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -737,6 +737,7 @@ class ImageLoaderCurrent implements ImageLoader {
       processPermission(in, v);
     } else if (numBlocks == -2) {
       v.visit(ImageElement.SYMLINK, Text.readString(in));
+      processPermission(in, v);
     } else if (numBlocks == -3) { // reference node
       final boolean isWithName = in.readBoolean();
       int snapshotId = in.readInt();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java

@@ -115,7 +115,7 @@ final class LsrPBImage {
         case INODE:
           loadINodeSection(is);
           break;
-        case INODE_REFRENCE:
+        case INODE_REFERENCE:
           loadINodeReferenceSection(is);
           break;
         case INODE_DIR:

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java

@@ -113,7 +113,7 @@ public final class PBImageXmlWriter {
         case INODE:
           dumpINodeSection(is);
           break;
-        case INODE_REFRENCE:
+        case INODE_REFERENCE:
           dumpINodeReferenceSection(is);
           break;
         case INODE_DIR:

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java

@@ -344,14 +344,15 @@ public class HftpFileSystem extends FileSystem
   }
 
   static class RangeHeaderUrlOpener extends ByteRangeInputStream.URLOpener {
-    URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_SYSTEM_CONNECTION_FACTORY;
+    private final URLConnectionFactory connFactory;
 
-    RangeHeaderUrlOpener(final URL url) {
+    RangeHeaderUrlOpener(URLConnectionFactory connFactory, final URL url) {
       super(url);
+      this.connFactory = connFactory;
     }
 
     protected HttpURLConnection openConnection() throws IOException {
-      return (HttpURLConnection)connectionFactory.openConnection(url);
+      return (HttpURLConnection)connFactory.openConnection(url);
     }
 
     /** Use HTTP Range header for specifying offset. */
@@ -381,8 +382,9 @@ public class HftpFileSystem extends FileSystem
       super(o, r);
     }
 
-    RangeHeaderInputStream(final URL url) {
-      this(new RangeHeaderUrlOpener(url), new RangeHeaderUrlOpener(null));
+    RangeHeaderInputStream(URLConnectionFactory connFactory, final URL url) {
+      this(new RangeHeaderUrlOpener(connFactory, url),
+          new RangeHeaderUrlOpener(connFactory, null));
     }
 
     @Override
@@ -397,7 +399,7 @@ public class HftpFileSystem extends FileSystem
     String path = "/data" + ServletUtil.encodePath(f.toUri().getPath());
     String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter());
     URL u = getNamenodeURL(path, query);
-    return new FSDataInputStream(new RangeHeaderInputStream(u));
+    return new FSDataInputStream(new RangeHeaderInputStream(connectionFactory, u));
   }
 
   @Override

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -74,7 +74,7 @@ public class TestBalancer {
 
   ClientProtocol client;
 
-  static final long TIMEOUT = 20000L; //msec
+  static final long TIMEOUT = 40000L; //msec
   static final double CAPACITY_ALLOWED_VARIANCE = 0.005;  // 0.5%
   static final double BALANCE_ALLOWED_VARIANCE = 0.11;    // 10%+delta
   static final int DEFAULT_BLOCK_SIZE = 10;

+ 16 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java

@@ -66,7 +66,7 @@ public class TestRBWBlockInvalidation {
    * datanode, namenode should ask to invalidate that corrupted block and
    * schedule replication for one more replica for that under replicated block.
    */
-  @Test(timeout=60000)
+  @Test(timeout=600000)
   public void testBlockInvalidationWhenRBWReplicaMissedInDN()
       throws IOException, InterruptedException {
     // This test cannot pass on Windows due to file locking enforcement.  It will
@@ -75,7 +75,7 @@ public class TestRBWBlockInvalidation {
 
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 2);
-    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 100);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 300);
     conf.setLong(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
@@ -104,23 +104,24 @@ public class TestRBWBlockInvalidation {
           metaFile.delete());
 
       out.close();
-
-      // Check datanode has reported the corrupt block.
-      int corruptReplicas = 0;
+      
+      int liveReplicas = 0;
       while (true) {
-        if ((corruptReplicas = countReplicas(namesystem, blk).corruptReplicas()) > 0) {
+        if ((liveReplicas = countReplicas(namesystem, blk).liveReplicas()) < 2) {
+          // This confirms we have a corrupt replica
+          LOG.info("Live Replicas after corruption: " + liveReplicas);
           break;
         }
         Thread.sleep(100);
       }
-      assertEquals("There should be 1 replica in the corruptReplicasMap", 1,
-          corruptReplicas);
-
-      // Check the block has got replicated to another datanode.
-      blk = DFSTestUtil.getFirstBlock(fs, testPath);
-      int liveReplicas = 0;
+      assertEquals("There should be less than 2 replicas in the "
+          + "liveReplicasMap", 1, liveReplicas);
+      
       while (true) {
-        if ((liveReplicas = countReplicas(namesystem, blk).liveReplicas()) > 1) {
+        if ((liveReplicas =
+              countReplicas(namesystem, blk).liveReplicas()) > 1) {
+          //Wait till the live replica count becomes equal to Replication Factor
+          LOG.info("Live Replicas after Rereplication: " + liveReplicas);
           break;
         }
         Thread.sleep(100);
@@ -128,9 +129,9 @@ public class TestRBWBlockInvalidation {
       assertEquals("There should be two live replicas", 2,
           liveReplicas);
 
-      // sleep for 1 second, so that by this time datanode reports the corrupt
+      // sleep for 2 seconds, so that by this time datanode reports the corrupt
       // block after a live replica of block got replicated.
-      Thread.sleep(1000);
+      Thread.sleep(2000);
 
       // Check that there is no corrupt block in the corruptReplicasMap.
       assertEquals("There should not be any replica in the corruptReplicasMap",

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java

@@ -97,12 +97,13 @@ public static class MockHttpURLConnection extends HttpURLConnection {
 
   @Test
   public void testByteRange() throws IOException {
+    URLConnectionFactory factory = mock(URLConnectionFactory.class);
     HftpFileSystem.RangeHeaderUrlOpener ospy = spy(
-        new HftpFileSystem.RangeHeaderUrlOpener(new URL("http://test/")));
+        new HftpFileSystem.RangeHeaderUrlOpener(factory, new URL("http://test/")));
     doReturn(new MockHttpURLConnection(ospy.getURL())).when(ospy)
         .openConnection();
     HftpFileSystem.RangeHeaderUrlOpener rspy = spy(
-        new HftpFileSystem.RangeHeaderUrlOpener((URL) null));
+        new HftpFileSystem.RangeHeaderUrlOpener(factory, (URL) null));
     doReturn(new MockHttpURLConnection(rspy.getURL())).when(rspy)
         .openConnection();
     ByteRangeInputStream is = new HftpFileSystem.RangeHeaderInputStream(ospy, rspy);
@@ -171,12 +172,15 @@ public static class MockHttpURLConnection extends HttpURLConnection {
       assertEquals("Should fail because incorrect response code was sent",
                    "HTTP_OK expected, received 206", e.getMessage());
     }
+    is.close();
   }
 
   @Test
   public void testPropagatedClose() throws IOException {
-    ByteRangeInputStream brs = spy(
-        new HftpFileSystem.RangeHeaderInputStream(new URL("http://test/")));
+    URLConnectionFactory factory = mock(URLConnectionFactory.class);
+
+    ByteRangeInputStream brs = spy(new HftpFileSystem.RangeHeaderInputStream(
+        factory, new URL("http://test/")));
 
     InputStream mockStream = mock(InputStream.class);
     doReturn(mockStream).when(brs).openInputStream();

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.web;
 
 import java.io.File;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.URI;
 
@@ -30,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -65,9 +67,11 @@ public class TestHttpsFileSystem {
 
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
-    cluster.getFileSystem().create(new Path("/test")).close();
+    OutputStream os = cluster.getFileSystem().create(new Path("/test"));
+    os.write(23);
+    os.close();
     InetSocketAddress addr = cluster.getNameNode().getHttpsAddress();
-    nnAddr = addr.getHostName() + ":" + addr.getPort();
+    nnAddr = NetUtils.getHostPortString(addr);
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, nnAddr);
   }
 
@@ -82,6 +86,9 @@ public class TestHttpsFileSystem {
   public void testHsftpFileSystem() throws Exception {
     FileSystem fs = FileSystem.get(new URI("hsftp://" + nnAddr), conf);
     Assert.assertTrue(fs.exists(new Path("/test")));
+    InputStream is = fs.open(new Path("/test"));
+    Assert.assertEquals(23, is.read());
+    is.close();
     fs.close();
   }
 

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

@@ -139,6 +139,18 @@ Trunk (Unreleased)
 
     MAPREDUCE-5717. Task pings are interpreted as task progress (jlowe)
 
+Release 2.5.0 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES 
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 15 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.hs;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -169,8 +170,21 @@ public class JobHistoryServer extends CompositeService {
   }
 
   protected void doSecureLogin(Configuration conf) throws IOException {
+    InetSocketAddress socAddr = getBindAddress(conf);
     SecurityUtil.login(conf, JHAdminConfig.MR_HISTORY_KEYTAB,
-        JHAdminConfig.MR_HISTORY_PRINCIPAL);
+        JHAdminConfig.MR_HISTORY_PRINCIPAL, socAddr.getHostName());
+  }
+
+  /**
+   * Retrieve JHS bind address from configuration
+   *
+   * @param conf
+   * @return InetSocketAddress
+   */
+  public static InetSocketAddress getBindAddress(Configuration conf) {
+    return conf.getSocketAddr(JHAdminConfig.MR_HISTORY_ADDRESS,
+      JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS,
+      JHAdminConfig.DEFAULT_MR_HISTORY_PORT);
   }
 
   @Override

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

@@ -15,6 +15,18 @@ Trunk - Unreleased
     YARN-524 TestYarnVersionInfo failing if generated properties doesn't
     include an SVN URL. (stevel)
 
+Release 2.5.0 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES 
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -277,6 +289,20 @@ Release 2.4.0 - UNRELEASED
     instead rely on the http policy framework. And also fix some bugs related
     to https handling in YARN web-apps. (Haohui Mai via vinodkv)
 
+    YARN-1721. When moving app between queues in Fair Scheduler, grab lock on
+    FSSchedulerApp (Sandy Ryza)
+
+    YARN-1724. Race condition in Fair Scheduler when continuous scheduling is
+    turned on (Sandy Ryza)
+
+    YARN-1590. Fixed ResourceManager, web-app proxy and MR JobHistoryServer to
+    expand _HOST properly in their kerberos principles. (Mohammad Kamrul Islam
+    va vinodkv)
+
+    YARN-1428. Fixed RM to write the final state of RMApp/RMAppAttempt to the 
+    application history store in the transition to the final state. (Contributed
+    by Zhijie Shen)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -861,6 +861,9 @@ public class YarnConfiguration extends Configuration {
   /** The address for the web proxy.*/
   public static final String PROXY_ADDRESS =
     PROXY_PREFIX + "address";
+  public static final int DEFAULT_PROXY_PORT = 9099;
+  public static final String DEFAULT_PROXY_ADDRESS =
+    "0.0.0.0:" + DEFAULT_RM_PORT;
   
   /**
    * YARN Service Level Authorization

+ 16 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -894,8 +895,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
   
   protected void doSecureLogin() throws IOException {
+	InetSocketAddress socAddr = getBindAddress(conf);
     SecurityUtil.login(this.conf, YarnConfiguration.RM_KEYTAB,
-        YarnConfiguration.RM_PRINCIPAL);
+        YarnConfiguration.RM_PRINCIPAL, socAddr.getHostName());
   }
 
   @Override
@@ -1042,4 +1044,17 @@ public class ResourceManager extends CompositeService implements Recoverable {
     addIfService(rmDispatcher);
     rmContext.setDispatcher(rmDispatcher);
   }
+
+
+  /**
+   * Retrieve RM bind address from configuration
+   *
+   * @param conf
+   * @return InetSocketAddress
+   */
+public static InetSocketAddress getBindAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_PORT);
+  }
 }

+ 10 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java

@@ -43,9 +43,12 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.records.Applicati
 import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 
 /**
@@ -219,12 +222,13 @@ public class RMApplicationHistoryWriter extends CompositeService {
   }
 
   @SuppressWarnings("unchecked")
-  public void applicationFinished(RMApp app) {
+  public void applicationFinished(RMApp app, RMAppState finalState) {
     dispatcher.getEventHandler().handle(
       new WritingApplicationFinishEvent(app.getApplicationId(),
         ApplicationFinishData.newInstance(app.getApplicationId(),
           app.getFinishTime(), app.getDiagnostics().toString(),
-          app.getFinalApplicationStatus(), app.createApplicationState())));
+          app.getFinalApplicationStatus(),
+          RMServerUtils.createApplicationState(finalState))));
   }
 
   @SuppressWarnings("unchecked")
@@ -239,15 +243,16 @@ public class RMApplicationHistoryWriter extends CompositeService {
   }
 
   @SuppressWarnings("unchecked")
-  public void applicationAttemptFinished(RMAppAttempt appAttempt) {
+  public void applicationAttemptFinished(RMAppAttempt appAttempt,
+      RMAppAttemptState finalState) {
     if (historyServiceEnabled) {
       dispatcher.getEventHandler().handle(
         new WritingApplicationAttemptFinishEvent(appAttempt.getAppAttemptId(),
           ApplicationAttemptFinishData.newInstance(
             appAttempt.getAppAttemptId(), appAttempt.getDiagnostics()
               .toString(), appAttempt.getTrackingUrl(), appAttempt
-              .getFinalApplicationStatus(), appAttempt
-              .createApplicationAttemptState())));
+              .getFinalApplicationStatus(),
+              RMServerUtils.createApplicationAttemptState(finalState))));
     }
   }
 

+ 21 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -112,9 +112,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   private long storedFinishTime = 0;
   private RMAppAttempt currentAttempt;
   private String queue;
-  @SuppressWarnings("rawtypes")
   private EventHandler handler;
-  private static final FinalTransition FINAL_TRANSITION = new FinalTransition();
   private static final AppFinishedTransition FINISHED_TRANSITION =
       new AppFinishedTransition();
 
@@ -705,7 +703,6 @@ public class RMAppImpl implements RMApp, Recoverable {
    * either as an exception for failure or null for success, or the client will
    * be left waiting forever.
    */
-  @SuppressWarnings("unchecked")
   private static final class RMAppMoveTransition extends RMAppTransition {
     public void transition(RMAppImpl app, RMAppEvent event) {
       RMAppMoveEvent moveEvent = (RMAppMoveEvent) event;
@@ -723,7 +720,6 @@ public class RMAppImpl implements RMApp, Recoverable {
     }
   }
 
-  @SuppressWarnings("unchecked")
   private static final class RMAppRecoveredTransition implements
       MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
 
@@ -742,7 +738,7 @@ public class RMAppImpl implements RMApp, Recoverable {
 
       // The app has completed.
       if (app.recoveredFinalState != null) {
-        FINAL_TRANSITION.transition(app, event);
+        new FinalTransition(app.recoveredFinalState).transition(app, event);
         return app.recoveredFinalState;
       }
 
@@ -824,7 +820,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       LOG.info(msg);
       app.diagnostics.append(msg);
       // Inform the node for app-finish
-      FINAL_TRANSITION.transition(app, event);
+      new FinalTransition(RMAppState.FAILED).transition(app, event);
     }
   }
 
@@ -937,6 +933,10 @@ public class RMAppImpl implements RMApp, Recoverable {
   }
 
   private static class AppFinishedTransition extends FinalTransition {
+    public AppFinishedTransition() {
+      super(RMAppState.FINISHED);
+    }
+
     public void transition(RMAppImpl app, RMAppEvent event) {
       RMAppFinishedAttemptEvent finishedEvent =
           (RMAppFinishedAttemptEvent)event;
@@ -980,6 +980,10 @@ public class RMAppImpl implements RMApp, Recoverable {
 
 
   private static class AppKilledTransition extends FinalTransition {
+    public AppKilledTransition() {
+      super(RMAppState.KILLED);
+    }
+
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
       app.diagnostics.append(getAppKilledDiagnostics());
@@ -1002,6 +1006,10 @@ public class RMAppImpl implements RMApp, Recoverable {
 
   private static final class AppRejectedTransition extends
       FinalTransition{
+    public AppRejectedTransition() {
+      super(RMAppState.FAILED);
+    }
+
     public void transition(RMAppImpl app, RMAppEvent event) {
       RMAppRejectedEvent rejectedEvent = (RMAppRejectedEvent)event;
       app.diagnostics.append(rejectedEvent.getMessage());
@@ -1011,6 +1019,12 @@ public class RMAppImpl implements RMApp, Recoverable {
 
   private static class FinalTransition extends RMAppTransition {
 
+    private final RMAppState finalState;
+
+    public FinalTransition(RMAppState finalState) {
+      this.finalState = finalState;
+    }
+
     private Set<NodeId> getNodesOnWhichAttemptRan(RMAppImpl app) {
       Set<NodeId> nodes = new HashSet<NodeId>();
       for (RMAppAttempt attempt : app.attempts.values()) {
@@ -1035,10 +1049,8 @@ public class RMAppImpl implements RMApp, Recoverable {
           new RMAppManagerEvent(app.applicationId,
           RMAppManagerEventType.APP_COMPLETED));
 
-      // TODO: We need to fix for the problem that RMApp enters the final state
-      // after RMAppAttempt in the killing case
       app.rmContext.getRMApplicationHistoryWriter()
-          .applicationFinished(app);
+          .applicationFinished(app, finalState);
     };
   }
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -1053,7 +1053,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       appAttempt.removeCredentials(appAttempt);
 
       appAttempt.rmContext.getRMApplicationHistoryWriter()
-          .applicationAttemptFinished(appAttempt);
+          .applicationAttemptFinished(appAttempt, finalAttemptState);
     }
   }
 

+ 28 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -989,7 +989,13 @@ public class FairScheduler extends AbstractYarnScheduler {
   private void continuousScheduling() {
     while (true) {
       List<NodeId> nodeIdList = new ArrayList<NodeId>(nodes.keySet());
-      Collections.sort(nodeIdList, nodeAvailableResourceComparator);
+      // Sort the nodes by space available on them, so that we offer
+      // containers on emptier nodes first, facilitating an even spread. This
+      // requires holding the scheduler lock, so that the space available on a
+      // node doesn't change during the sort.
+      synchronized (this) {
+        Collections.sort(nodeIdList, nodeAvailableResourceComparator);
+      }
 
       // iterate all nodes
       for (NodeId nodeId : nodeIdList) {
@@ -1366,24 +1372,26 @@ public class FairScheduler extends AbstractYarnScheduler {
       throw new YarnException("App to be moved " + appId + " not found.");
     }
     FSSchedulerApp attempt = (FSSchedulerApp) app.getCurrentAppAttempt();
-    
-    FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
-    FSLeafQueue targetQueue = queueMgr.getLeafQueue(queueName, false);
-    if (targetQueue == null) {
-      throw new YarnException("Target queue " + queueName
-          + " not found or is not a leaf queue.");
-    }
-    if (targetQueue == oldQueue) {
-      return oldQueue.getQueueName();
-    }
-    
-    if (oldQueue.getRunnableAppSchedulables().contains(
-        attempt.getAppSchedulable())) {
-      verifyMoveDoesNotViolateConstraints(attempt, oldQueue, targetQueue);
+    // To serialize with FairScheduler#allocate, synchronize on app attempt
+    synchronized (attempt) {
+      FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
+      FSLeafQueue targetQueue = queueMgr.getLeafQueue(queueName, false);
+      if (targetQueue == null) {
+        throw new YarnException("Target queue " + queueName
+            + " not found or is not a leaf queue.");
+      }
+      if (targetQueue == oldQueue) {
+        return oldQueue.getQueueName();
+      }
+      
+      if (oldQueue.getRunnableAppSchedulables().contains(
+          attempt.getAppSchedulable())) {
+        verifyMoveDoesNotViolateConstraints(attempt, oldQueue, targetQueue);
+      }
+      
+      executeMove(app, attempt, oldQueue, targetQueue);
+      return targetQueue.getQueueName();
     }
-    
-    executeMove(app, attempt, oldQueue, targetQueue);
-    return targetQueue.getQueueName();
   }
   
   private void verifyMoveDoesNotViolateConstraints(FSSchedulerApp app,
@@ -1420,8 +1428,8 @@ public class FairScheduler extends AbstractYarnScheduler {
   }
   
   /**
-   * Helper for moveApplication, which is synchronized, so all operations will
-   * be atomic.
+   * Helper for moveApplication, which has appropriate synchronization, so all
+   * operations will be atomic.
    */
   private void executeMove(SchedulerApplication app, FSSchedulerApp attempt,
       FSLeafQueue oldQueue, FSLeafQueue newQueue) {

+ 8 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java

@@ -137,8 +137,6 @@ public class TestRMApplicationHistoryWriter {
       new StringBuilder("test diagnostics info"));
     when(app.getFinalApplicationStatus()).thenReturn(
       FinalApplicationStatus.UNDEFINED);
-    when(app.createApplicationState())
-      .thenReturn(YarnApplicationState.FINISHED);
     return app;
   }
 
@@ -156,8 +154,6 @@ public class TestRMApplicationHistoryWriter {
     when(appAttempt.getTrackingUrl()).thenReturn("test url");
     when(appAttempt.getFinalApplicationStatus()).thenReturn(
       FinalApplicationStatus.UNDEFINED);
-    when(appAttempt.createApplicationAttemptState()).thenReturn(
-      YarnApplicationAttemptState.FINISHED);
     return appAttempt;
   }
 
@@ -200,7 +196,7 @@ public class TestRMApplicationHistoryWriter {
     Assert.assertEquals(0L, appHD.getSubmitTime());
     Assert.assertEquals(1L, appHD.getStartTime());
 
-    writer.applicationFinished(app);
+    writer.applicationFinished(app, RMAppState.FINISHED);
     for (int i = 0; i < MAX_RETRIES; ++i) {
       appHD = store.getApplication(ApplicationId.newInstance(0, 1));
       if (appHD.getYarnApplicationState() != null) {
@@ -241,7 +237,7 @@ public class TestRMApplicationHistoryWriter {
       ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1), 1),
       appAttemptHD.getMasterContainerId());
 
-    writer.applicationAttemptFinished(appAttempt);
+    writer.applicationAttemptFinished(appAttempt, RMAppAttemptState.FINISHED);
     for (int i = 0; i < MAX_RETRIES; ++i) {
       appAttemptHD =
           store.getApplicationAttempt(ApplicationAttemptId.newInstance(
@@ -326,9 +322,10 @@ public class TestRMApplicationHistoryWriter {
           writer.containerStarted(container);
           writer.containerFinished(container);
         }
-        writer.applicationAttemptFinished(appAttempt);
+        writer.applicationAttemptFinished(
+            appAttempt, RMAppAttemptState.FINISHED);
       }
-      writer.applicationFinished(app);
+      writer.applicationFinished(app, RMAppState.FINISHED);
     }
     for (int i = 0; i < MAX_RETRIES; ++i) {
       if (allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)) {
@@ -369,7 +366,7 @@ public class TestRMApplicationHistoryWriter {
           }
 
           @Override
-          public void applicationFinished(RMApp app) {
+          public void applicationFinished(RMApp app, RMAppState finalState) {
           }
 
           @Override
@@ -377,7 +374,8 @@ public class TestRMApplicationHistoryWriter {
           }
 
           @Override
-          public void applicationAttemptFinished(RMAppAttempt appAttempt) {
+          public void applicationAttemptFinished(
+              RMAppAttempt appAttempt, RMAppAttemptState finalState) {
           }
 
           @Override

+ 21 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessM
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -71,6 +70,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.mockito.ArgumentCaptor;
 
 
 @RunWith(value = Parameterized.class)
@@ -308,16 +308,6 @@ public class TestRMAppTransitions {
         "Application killed by user.", diag.toString());
   }
 
-  private void assertAppAndAttemptKilled(RMApp application)
-      throws InterruptedException {
-    sendAttemptUpdateSavedEvent(application);
-    sendAppUpdateSavedEvent(application);
-    assertKilled(application);
-    Assert.assertEquals(RMAppAttemptState.KILLED, application
-      .getCurrentAppAttempt().getAppAttemptState());
-    assertAppFinalStateSaved(application);
-  }
-
   private void assertFailed(RMApp application, String regex) {
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FAILED, application);
@@ -511,7 +501,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertKilled(application);
     assertAppFinalStateNotSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.KILLED);
   }
 
   @Test
@@ -528,7 +518,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertFailed(application, rejectedText);
     assertAppFinalStateNotSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.FAILED);
   }
 
   @Test (timeout = 30000)
@@ -543,7 +533,7 @@ public class TestRMAppTransitions {
     rmDispatcher.await();
     sendAppUpdateSavedEvent(application);
     assertKilled(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.KILLED);
   }
 
   @Test (timeout = 30000)
@@ -560,7 +550,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.FAILED);
   }
 
   @Test (timeout = 30000)
@@ -577,7 +567,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.FAILED);
   }
 
   @Test
@@ -592,7 +582,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertKilled(application);
     assertAppFinalStateSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.KILLED);
   }
 
   @Test
@@ -627,7 +617,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertFailed(application, ".*" + message + ".*Failing the application.*");
     assertAppFinalStateSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.FAILED);
   }
 
   @Test
@@ -649,7 +639,7 @@ public class TestRMAppTransitions {
     sendAppUpdateSavedEvent(application);
     assertKilled(application);
     assertAppFinalStateSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.KILLED);
   }
 
   @Test
@@ -672,7 +662,7 @@ public class TestRMAppTransitions {
     sendAttemptUpdateSavedEvent(application);
     sendAppUpdateSavedEvent(application);
     assertKilled(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.KILLED);
   }
 
   @Test
@@ -727,7 +717,7 @@ public class TestRMAppTransitions {
     rmDispatcher.await();
     assertFailed(application, ".*Failing the application.*");
     assertAppFinalStateSaved(application);
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.FAILED);
   }
 
   @Test
@@ -785,7 +775,7 @@ public class TestRMAppTransitions {
     StringBuilder diag = application.getDiagnostics();
     Assert.assertEquals("application diagnostics is not correct",
         "", diag.toString());
-    verify(writer).applicationFinished(any(RMApp.class));
+    verifyApplicationFinished(RMAppState.FINISHED);
   }
 
   @Test (timeout = 30000)
@@ -810,10 +800,10 @@ public class TestRMAppTransitions {
     rmDispatcher.await();
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FAILED, application);
+    verifyApplicationFinished(RMAppState.FAILED);
 
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FAILED, application);
-    verify(writer).applicationFinished(any(RMApp.class));
   }
 
   @Test (timeout = 30000)
@@ -856,10 +846,10 @@ public class TestRMAppTransitions {
     rmDispatcher.await();
     assertTimesAtFinish(application);
     assertAppState(RMAppState.KILLED, application);
+    verifyApplicationFinished(RMAppState.KILLED);
 
     assertTimesAtFinish(application);
     assertAppState(RMAppState.KILLED, application);
-    verify(writer).applicationFinished(any(RMApp.class));
   }
 
   @Test
@@ -871,4 +861,11 @@ public class TestRMAppTransitions {
     report = app.createAndGetApplicationReport("clientuser", true);
     Assert.assertNotNull(report.getApplicationResourceUsageReport());
   }
+
+  private void verifyApplicationFinished(RMAppState state) {
+    ArgumentCaptor<RMAppState> finalState =
+        ArgumentCaptor.forClass(RMAppState.class);
+    verify(writer).applicationFinished(any(RMApp.class), finalState.capture());
+    Assert.assertEquals(state, finalState.getValue());
+  }
 }

+ 21 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -103,6 +103,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.mockito.ArgumentCaptor;
 
 @RunWith(value = Parameterized.class)
 public class TestRMAppAttemptTransitions {
@@ -367,6 +368,7 @@ public class TestRMAppAttemptTransitions {
     // verify(application).handle(anyObject());
     verify(application).handle(any(RMAppRejectedEvent.class));
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
   /**
@@ -384,9 +386,9 @@ public class TestRMAppAttemptTransitions {
     assertEquals(0, applicationAttempt.getRanNodes().size());
     assertNull(applicationAttempt.getFinalApplicationStatus());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
-    verify(writer).applicationAttemptFinished(any(RMAppAttempt.class));
     verifyAttemptFinalStateSaved();
     assertFalse(transferStateFromPreviousAttempt);
+    verifyApplicationAttemptFinished(RMAppAttemptState.KILLED);
   }
   
   /**
@@ -460,8 +462,8 @@ public class TestRMAppAttemptTransitions {
     // Check events
     verify(application, times(1)).handle(any(RMAppFailedAttemptEvent.class));
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
-    verify(writer).applicationAttemptFinished(any(RMAppAttempt.class));
     verifyAttemptFinalStateSaved();
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
   /**
@@ -496,7 +498,6 @@ public class TestRMAppAttemptTransitions {
       assertEquals(getProxyUrl(applicationAttempt), 
           applicationAttempt.getTrackingUrl());
     }
-    verify(writer).applicationAttemptStarted(any(RMAppAttempt.class));
     // TODO - need to add more checks relevant to this state
   }
 
@@ -544,6 +545,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
     assertFalse(transferStateFromPreviousAttempt);
+    verifyApplicationAttemptFinished(RMAppAttemptState.FINISHED);
   }
   
   
@@ -806,7 +808,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(RMAppAttemptState.FAILED,
       applicationAttempt.getAppAttemptState());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
-    verify(writer).applicationAttemptFinished(any(RMAppAttempt.class));
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
   
   @Test
@@ -846,6 +848,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(rmAppPageUrl, applicationAttempt.getOriginalTrackingUrl());
     assertEquals(rmAppPageUrl, applicationAttempt.getTrackingUrl());
     verifyAMHostAndPortInvalidated();
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
   @Test
@@ -883,6 +886,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(rmAppPageUrl, applicationAttempt.getTrackingUrl());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
     verifyAMHostAndPortInvalidated();
+    verifyApplicationAttemptFinished(RMAppAttemptState.KILLED);
   }
 
   @Test(timeout=10000)
@@ -903,6 +907,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(rmAppPageUrl, applicationAttempt.getOriginalTrackingUrl());
     assertEquals(rmAppPageUrl, applicationAttempt.getTrackingUrl());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
   @Test(timeout=20000)
@@ -925,6 +930,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(rmAppPageUrl, applicationAttempt.getTrackingUrl());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
     verifyAMHostAndPortInvalidated();
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
   @Test 
@@ -1177,6 +1183,7 @@ public class TestRMAppAttemptTransitions {
       applicationAttempt.getAppAttemptState());
     // should not kill containers when attempt fails.
     assertTrue(transferStateFromPreviousAttempt);
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
 
     // failed attempt captured the container finished event.
     assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
@@ -1216,6 +1223,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(RMAppAttemptState.FAILED,
       applicationAttempt.getAppAttemptState());
     assertFalse(transferStateFromPreviousAttempt);
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
   private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
@@ -1245,4 +1253,13 @@ public class TestRMAppAttemptTransitions {
     assertEquals("N/A", applicationAttempt.getHost());
     assertEquals(-1, applicationAttempt.getRpcPort());
   }
+
+  private void verifyApplicationAttemptFinished(RMAppAttemptState state) {
+    ArgumentCaptor<RMAppAttemptState> finalState =
+        ArgumentCaptor.forClass(RMAppAttemptState.class);
+    verify(writer).applicationAttemptFinished(
+        any(RMAppAttempt.class), finalState.capture());
+    Assert.assertEquals(state, finalState.getValue());
+  }
+
 }

+ 15 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.webproxy;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -69,8 +70,21 @@ public class WebAppProxyServer extends CompositeService {
    * @throws IOException on any error.
    */
   protected void doSecureLogin(Configuration conf) throws IOException {
+    InetSocketAddress socAddr = getBindAddress(conf);  
     SecurityUtil.login(conf, YarnConfiguration.PROXY_KEYTAB,
-        YarnConfiguration.PROXY_PRINCIPAL);
+        YarnConfiguration.PROXY_PRINCIPAL, socAddr.getHostName());
+  }
+
+  /**
+   * Retrieve PROXY bind address from configuration
+   *
+   * @param conf
+   * @return InetSocketAddress
+   */
+  public static InetSocketAddress getBindAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.PROXY_ADDRESS,
+      YarnConfiguration.DEFAULT_PROXY_ADDRESS,
+      YarnConfiguration.DEFAULT_PROXY_PORT);
   }
 
   public static void main(String[] args) {