Browse Source

Merging r1524587 through r1525408 from trunk to branch HDFS-2832.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1525410 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 12 năm trước cách đây
mục cha
commit
5133d78c1a
56 tập tin đã thay đổi với 1596 bổ sung112 xóa
  1. 17 2
      BUILDING.txt
  2. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 5 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
  4. 1 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java
  5. 8 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
  6. 39 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
  7. 3 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
  8. 55 3
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java
  9. 2 3
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
  10. 2 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
  11. 1 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServerHandler.java
  12. 1 1
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServerHandler.java
  13. 2 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
  14. 4 2
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java
  15. 1 2
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java
  16. 4 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
  17. 6 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
  19. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation-background.gif
  20. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation.gif
  21. 79 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
  22. 14 0
      hadoop-mapreduce-project/CHANGES.txt
  23. 6 0
      hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
  24. 15 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  25. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
  26. 14 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  27. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
  28. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
  29. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
  30. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  31. 39 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/WebAppUtil.java
  32. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/AMAttemptInfo.java
  33. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  34. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRConfig.java
  35. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  36. 16 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  37. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
  38. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
  39. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
  40. 37 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  41. 42 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
  42. 11 0
      hadoop-yarn-project/CHANGES.txt
  43. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
  44. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
  45. 32 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  46. 45 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  47. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
  48. 0 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
  49. 11 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  50. 621 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
  51. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
  52. 70 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStore.java
  53. 218 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStoreZKClientConnections.java
  54. 57 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestEmptyQueues.java
  55. 49 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
  56. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java

+ 17 - 2
BUILDING.txt

@@ -4,8 +4,8 @@ Build instructions for Hadoop
 Requirements:
 Requirements:
 
 
 * Unix System
 * Unix System
-* JDK 1.6
-* Maven 3.0
+* JDK 1.6+
+* Maven 3.0 or later
 * Findbugs 1.3.9 (if running findbugs)
 * Findbugs 1.3.9 (if running findbugs)
 * ProtocolBuffer 2.5.0
 * ProtocolBuffer 2.5.0
 * CMake 2.6 or newer (if compiling native code)
 * CMake 2.6 or newer (if compiling native code)
@@ -149,6 +149,21 @@ Create a local staging version of the website (in /tmp/hadoop-site)
 
 
 ----------------------------------------------------------------------------------
 ----------------------------------------------------------------------------------
 
 
+Handling out of memory errors in builds
+
+----------------------------------------------------------------------------------
+
+If the build process fails with an out of memory error, you should be able to fix
+it by increasing the memory used by maven -which can be done via the environment
+variable MAVEN_OPTS.
+
+Here is an example setting to allocate between 256 and 512 MB of heap space to
+Maven
+
+export MAVEN_OPTS="-Xms256m -Xmx512m"
+
+----------------------------------------------------------------------------------
+
 Building on OS/X
 Building on OS/X
 
 
 ----------------------------------------------------------------------------------
 ----------------------------------------------------------------------------------

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

@@ -366,6 +366,12 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9350. Hadoop not building against Java7 on OSX
     HADOOP-9350. Hadoop not building against Java7 on OSX
     (Robert Kanter via stevel)
     (Robert Kanter via stevel)
 
 
+    HADOOP-9929. Insufficient permissions for a path reported as file not found.
+    (Contributed by Colin Patrick McCabe)
+
+    HADOOP-9791. Add a test case covering long paths for new FileUtil access
+    check methods (ivanmi)
+
 Release 2.2.0 - UNRELEASED
 Release 2.2.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 5 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
@@ -50,26 +51,26 @@ class Globber {
     this.filter = filter;
     this.filter = filter;
   }
   }
 
 
-  private FileStatus getFileStatus(Path path) {
+  private FileStatus getFileStatus(Path path) throws IOException {
     try {
     try {
       if (fs != null) {
       if (fs != null) {
         return fs.getFileStatus(path);
         return fs.getFileStatus(path);
       } else {
       } else {
         return fc.getFileStatus(path);
         return fc.getFileStatus(path);
       }
       }
-    } catch (IOException e) {
+    } catch (FileNotFoundException e) {
       return null;
       return null;
     }
     }
   }
   }
 
 
-  private FileStatus[] listStatus(Path path) {
+  private FileStatus[] listStatus(Path path) throws IOException {
     try {
     try {
       if (fs != null) {
       if (fs != null) {
         return fs.listStatus(path);
         return fs.listStatus(path);
       } else {
       } else {
         return fc.util().listStatus(path);
         return fc.util().listStatus(path);
       }
       }
-    } catch (IOException e) {
+    } catch (FileNotFoundException e) {
       return new FileStatus[0];
       return new FileStatus[0];
     }
     }
   }
   }

+ 1 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java

@@ -17,7 +17,6 @@
  */
  */
 package org.apache.hadoop.http;
 package org.apache.hadoop.http;
 
 
-import com.google.common.annotations.VisibleForTesting;
 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.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -38,8 +37,7 @@ public class HttpConfig {
         CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_DEFAULT);
         CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_DEFAULT);
   }
   }
 
 
-  @VisibleForTesting
-  static void setSecure(boolean secure) {
+  public static void setSecure(boolean secure) {
     sslEnabled = secure;
     sslEnabled = secure;
   }
   }
 
 

+ 8 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java

@@ -53,6 +53,8 @@ import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
 /**
  * Copy-paste of ClientBase from ZooKeeper, but without any of the
  * Copy-paste of ClientBase from ZooKeeper, but without any of the
  * JMXEnv verification. There seems to be a bug ZOOKEEPER-1438
  * JMXEnv verification. There seems to be a bug ZOOKEEPER-1438
@@ -111,7 +113,9 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
         synchronized boolean isConnected() {
         synchronized boolean isConnected() {
             return connected;
             return connected;
         }
         }
-        synchronized void waitForConnected(long timeout) throws InterruptedException, TimeoutException {
+        @VisibleForTesting
+        public synchronized void waitForConnected(long timeout)
+            throws InterruptedException, TimeoutException {
             long expire = Time.now() + timeout;
             long expire = Time.now() + timeout;
             long left = timeout;
             long left = timeout;
             while(!connected && left > 0) {
             while(!connected && left > 0) {
@@ -123,7 +127,9 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
 
 
             }
             }
         }
         }
-        synchronized void waitForDisconnected(long timeout) throws InterruptedException, TimeoutException {
+        @VisibleForTesting
+        public synchronized void waitForDisconnected(long timeout)
+            throws InterruptedException, TimeoutException {
             long expire = Time.now() + timeout;
             long expire = Time.now() + timeout;
             long left = timeout;
             long left = timeout;
             while(connected && left > 0) {
             while(connected && left > 0) {

+ 39 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

@@ -254,6 +254,45 @@ public class TestNativeIO {
     File testFile = new File(TEST_DIR, "testfileaccess");
     File testFile = new File(TEST_DIR, "testfileaccess");
     assertTrue(testFile.createNewFile());
     assertTrue(testFile.createNewFile());
 
 
+    // Validate ACCESS_READ
+    FileUtil.setReadable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_READ));
+
+    FileUtil.setReadable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_READ));
+
+    // Validate ACCESS_WRITE
+    FileUtil.setWritable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_WRITE));
+
+    FileUtil.setWritable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_WRITE));
+
+    // Validate ACCESS_EXECUTE
+    FileUtil.setExecutable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_EXECUTE));
+
+    FileUtil.setExecutable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_EXECUTE));
+
+    // Validate that access checks work as expected for long paths
+
+    // Assemble a path longer then 260 chars (MAX_PATH)
+    String testFileRelativePath = "";
+    for (int i = 0; i < 15; ++i) {
+      testFileRelativePath += "testfileaccessfolder\\";
+    }
+    testFileRelativePath += "testfileaccess";
+    testFile = new File(TEST_DIR, testFileRelativePath);
+    assertTrue(testFile.getParentFile().mkdirs());
+    assertTrue(testFile.createNewFile());
+
     // Validate ACCESS_READ
     // Validate ACCESS_READ
     FileUtil.setReadable(testFile, false);
     FileUtil.setReadable(testFile, false);
     assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
     assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),

+ 3 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.nfs.nfs3;
 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.mount.MountdBase;
 import org.apache.hadoop.mount.MountdBase;
-import org.apache.hadoop.oncrpc.RpcFrameDecoder;
 import org.apache.hadoop.oncrpc.RpcProgram;
 import org.apache.hadoop.oncrpc.RpcProgram;
+import org.apache.hadoop.oncrpc.RpcUtil;
 import org.apache.hadoop.oncrpc.SimpleTcpServer;
 import org.apache.hadoop.oncrpc.SimpleTcpServer;
 import org.apache.hadoop.oncrpc.SimpleTcpServerHandler;
 import org.apache.hadoop.oncrpc.SimpleTcpServerHandler;
 import org.apache.hadoop.portmap.PortmapMapping;
 import org.apache.hadoop.portmap.PortmapMapping;
@@ -68,7 +68,8 @@ public abstract class Nfs3Base {
         return new ChannelPipelineFactory() {
         return new ChannelPipelineFactory() {
           @Override
           @Override
           public ChannelPipeline getPipeline() {
           public ChannelPipeline getPipeline() {
-            return Channels.pipeline(new RpcFrameDecoder(),
+            return Channels.pipeline(
+                RpcUtil.constructRpcFrameDecoder(),
                 new SimpleTcpServerHandler(rpcProgram));
                 new SimpleTcpServerHandler(rpcProgram));
           }
           }
         };
         };

+ 55 - 3
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java

@@ -17,13 +17,65 @@
  */
  */
 package org.apache.hadoop.oncrpc;
 package org.apache.hadoop.oncrpc;
 
 
-/**
- * The XID in RPC call. It is used for starting with new seed after each reboot.
- */
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.frame.FrameDecoder;
+
 public class RpcUtil {
 public class RpcUtil {
+  /**
+   * The XID in RPC call. It is used for starting with new seed after each reboot.
+   */
   private static int xid = (int) (System.currentTimeMillis() / 1000) << 12;
   private static int xid = (int) (System.currentTimeMillis() / 1000) << 12;
 
 
   public static int getNewXid(String caller) {
   public static int getNewXid(String caller) {
     return xid = ++xid + caller.hashCode();
     return xid = ++xid + caller.hashCode();
   }
   }
+
+  public static FrameDecoder constructRpcFrameDecoder() {
+    return new RpcFrameDecoder();
+  }
+
+  static class RpcFrameDecoder extends FrameDecoder {
+    public static final Log LOG = LogFactory.getLog(RpcFrameDecoder.class);
+    private ChannelBuffer currentFrame;
+
+    @Override
+    protected Object decode(ChannelHandlerContext ctx, Channel channel,
+        ChannelBuffer buf) {
+
+      if (buf.readableBytes() < 4)
+        return null;
+
+      buf.markReaderIndex();
+
+      byte[] fragmentHeader = new byte[4];
+      buf.readBytes(fragmentHeader);
+      int length = XDR.fragmentSize(fragmentHeader);
+      boolean isLast = XDR.isLastFragment(fragmentHeader);
+
+      if (buf.readableBytes() < length) {
+        buf.resetReaderIndex();
+        return null;
+      }
+
+      ChannelBuffer newFragment = buf.readSlice(length);
+      if (currentFrame == null) {
+        currentFrame = newFragment;
+      } else {
+        currentFrame = ChannelBuffers.wrappedBuffer(currentFrame, newFragment);
+      }
+
+      if (isLast) {
+        ChannelBuffer completeFrame = currentFrame;
+        currentFrame = null;
+        return completeFrame;
+      } else {
+        return null;
+      }
+    }
+  }
 }
 }

+ 2 - 3
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.oncrpc;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Executors;
 
 
-import org.apache.hadoop.oncrpc.RpcFrameDecoder;
-import org.apache.hadoop.oncrpc.XDR;
 import org.jboss.netty.bootstrap.ClientBootstrap;
 import org.jboss.netty.bootstrap.ClientBootstrap;
 import org.jboss.netty.channel.ChannelFactory;
 import org.jboss.netty.channel.ChannelFactory;
 import org.jboss.netty.channel.ChannelFuture;
 import org.jboss.netty.channel.ChannelFuture;
@@ -55,7 +53,8 @@ public class SimpleTcpClient {
     this.pipelineFactory = new ChannelPipelineFactory() {
     this.pipelineFactory = new ChannelPipelineFactory() {
       @Override
       @Override
       public ChannelPipeline getPipeline() {
       public ChannelPipeline getPipeline() {
-        return Channels.pipeline(new RpcFrameDecoder(),
+        return Channels.pipeline(
+            RpcUtil.constructRpcFrameDecoder(),
             new SimpleTcpClientHandler(request));
             new SimpleTcpClientHandler(request));
       }
       }
     };
     };

+ 2 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java

@@ -57,7 +57,8 @@ public class SimpleTcpServer {
     return new ChannelPipelineFactory() {
     return new ChannelPipelineFactory() {
       @Override
       @Override
       public ChannelPipeline getPipeline() {
       public ChannelPipeline getPipeline() {
-        return Channels.pipeline(new RpcFrameDecoder(),
+        return Channels.pipeline(
+            RpcUtil.constructRpcFrameDecoder(),
             new SimpleTcpServerHandler(rpcProgram));
             new SimpleTcpServerHandler(rpcProgram));
       }
       }
     };
     };

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServerHandler.java

@@ -44,7 +44,7 @@ public class SimpleTcpServerHandler extends SimpleChannelHandler {
   @Override
   @Override
   public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
   public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
     ChannelBuffer buf = (ChannelBuffer) e.getMessage();
     ChannelBuffer buf = (ChannelBuffer) e.getMessage();
-    XDR request = new XDR(buf.array());
+    XDR request = new XDR(buf.toByteBuffer().asReadOnlyBuffer(), XDR.State.READING);
     
     
     InetAddress remoteInetAddr = ((InetSocketAddress) ctx.getChannel()
     InetAddress remoteInetAddr = ((InetSocketAddress) ctx.getChannel()
         .getRemoteAddress()).getAddress();
         .getRemoteAddress()).getAddress();

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServerHandler.java

@@ -43,7 +43,7 @@ public class SimpleUdpServerHandler extends SimpleChannelHandler {
   public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
   public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
     ChannelBuffer buf = (ChannelBuffer) e.getMessage();
     ChannelBuffer buf = (ChannelBuffer) e.getMessage();
 
 
-    XDR request = new XDR(buf.array());
+    XDR request = new XDR(buf.toByteBuffer().asReadOnlyBuffer(), XDR.State.READING);
     
     
     InetAddress remoteInetAddr = ((InetSocketAddress) e.getRemoteAddress())
     InetAddress remoteInetAddr = ((InetSocketAddress) e.getRemoteAddress())
         .getAddress();
         .getAddress();

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java

@@ -46,7 +46,7 @@ public final class XDR {
 
 
   private ByteBuffer buf;
   private ByteBuffer buf;
 
 
-  private enum State {
+  public enum State {
     READING, WRITING,
     READING, WRITING,
   }
   }
 
 
@@ -66,7 +66,7 @@ public final class XDR {
     this(DEFAULT_INITIAL_CAPACITY);
     this(DEFAULT_INITIAL_CAPACITY);
   }
   }
 
 
-  private XDR(ByteBuffer buf, State state) {
+  public XDR(ByteBuffer buf, State state) {
     this.buf = buf;
     this.buf = buf;
     this.state = state;
     this.state = state;
   }
   }

+ 4 - 2
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java

@@ -18,12 +18,14 @@
 
 
 package org.apache.hadoop.oncrpc;
 package org.apache.hadoop.oncrpc;
 
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 
 
+import org.apache.hadoop.oncrpc.RpcUtil.RpcFrameDecoder;
 import org.apache.hadoop.oncrpc.security.CredentialsNone;
 import org.apache.hadoop.oncrpc.security.CredentialsNone;
 import org.apache.hadoop.oncrpc.security.VerifierNone;
 import org.apache.hadoop.oncrpc.security.VerifierNone;
 import org.jboss.netty.buffer.ByteBufferBackedChannelBuffer;
 import org.jboss.netty.buffer.ByteBufferBackedChannelBuffer;
@@ -138,7 +140,7 @@ public class TestFrameDecoder {
         buf);
         buf);
     assertTrue(channelBuffer != null);
     assertTrue(channelBuffer != null);
     // Complete frame should have to total size 10+10=20
     // Complete frame should have to total size 10+10=20
-    assertTrue(channelBuffer.array().length == 20);
+    assertEquals(20, channelBuffer.readableBytes());
   }
   }
 
 
   @Test
   @Test
@@ -195,4 +197,4 @@ public class TestFrameDecoder {
    * static void testDump() { XDR xdr_out = new XDR();
    * static void testDump() { XDR xdr_out = new XDR();
    * createPortmapXDRheader(xdr_out, 4); testRequest(xdr_out); }
    * createPortmapXDRheader(xdr_out, 4); testRequest(xdr_out); }
    */
    */
-}
+}

+ 1 - 2
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestXDR.java

@@ -17,10 +17,9 @@
  */
  */
 package org.apache.hadoop.oncrpc;
 package org.apache.hadoop.oncrpc;
 
 
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import junit.framework.Assert;
-
 public class TestXDR {
 public class TestXDR {
   private void serializeInt(int times) {
   private void serializeInt(int times) {
     XDR w = new XDR();
     XDR w = new XDR();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java

@@ -33,8 +33,8 @@ import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
 import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
 import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
 import org.apache.hadoop.oncrpc.RegistrationClient;
 import org.apache.hadoop.oncrpc.RegistrationClient;
 import org.apache.hadoop.oncrpc.RpcCall;
 import org.apache.hadoop.oncrpc.RpcCall;
-import org.apache.hadoop.oncrpc.RpcFrameDecoder;
 import org.apache.hadoop.oncrpc.RpcReply;
 import org.apache.hadoop.oncrpc.RpcReply;
+import org.apache.hadoop.oncrpc.RpcUtil;
 import org.apache.hadoop.oncrpc.SimpleTcpClient;
 import org.apache.hadoop.oncrpc.SimpleTcpClient;
 import org.apache.hadoop.oncrpc.SimpleTcpClientHandler;
 import org.apache.hadoop.oncrpc.SimpleTcpClientHandler;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.XDR;
@@ -136,8 +136,9 @@ public class TestOutOfOrderWrite {
     protected ChannelPipelineFactory setPipelineFactory() {
     protected ChannelPipelineFactory setPipelineFactory() {
       this.pipelineFactory = new ChannelPipelineFactory() {
       this.pipelineFactory = new ChannelPipelineFactory() {
         public ChannelPipeline getPipeline() {
         public ChannelPipeline getPipeline() {
-          return Channels.pipeline(new RpcFrameDecoder(), new WriteHandler(
-              request));
+          return Channels.pipeline(
+              RpcUtil.constructRpcFrameDecoder(),
+              new WriteHandler(request));
         }
         }
       };
       };
       return this.pipelineFactory;
       return this.pipelineFactory;

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

@@ -366,6 +366,9 @@ Release 2.1.1-beta - 2013-09-23
 
 
     HDFS-5199 Add more debug trace for NFS READ and WRITE. (brandonli)
     HDFS-5199 Add more debug trace for NFS READ and WRITE. (brandonli)
 
 
+    HDFS-5234 Move RpcFrameDecoder out of the public API.
+    (Haohui Mai via 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
@@ -479,6 +482,9 @@ Release 2.1.1-beta - 2013-09-23
     HDFS-5219. Add configuration keys for retry policy in WebHDFSFileSystem.
     HDFS-5219. Add configuration keys for retry policy in WebHDFSFileSystem.
     (Haohui Mai via jing9)
     (Haohui Mai via jing9)
 
 
+    HDFS-5231. Fix broken links in the document of HDFS Federation. (Haohui Mai
+    via jing9)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm

@@ -28,7 +28,7 @@ HDFS Federation
 
 
 * {Background}
 * {Background}
 
 
-[./federation-background.gif] HDFS Layers
+[./images/federation-background.gif] HDFS Layers
 
 
   HDFS has two main layers:
   HDFS has two main layers:
 
 
@@ -72,7 +72,7 @@ HDFS Federation
   send periodic heartbeats and block reports and handles commands from the 
   send periodic heartbeats and block reports and handles commands from the 
   Namenodes.
   Namenodes.
 
 
-[./federation.gif] HDFS Federation Architecture
+[./images/federation.gif] HDFS Federation Architecture
 
 
 
 
   <<Block Pool>>
   <<Block Pool>>

+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/federation-background.gif → hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation-background.gif


+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/federation.gif → hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation.gif


+ 79 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
@@ -27,10 +28,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.*;
 import org.junit.*;
 
 
 public class TestGlobPaths {
 public class TestGlobPaths {
 
 
+  private static final UserGroupInformation unprivilegedUser =
+      UserGroupInformation.createRemoteUser("myuser");
+
   static class RegexPathFilter implements PathFilter {
   static class RegexPathFilter implements PathFilter {
 
 
     private final String regex;
     private final String regex;
@@ -47,17 +53,33 @@ public class TestGlobPaths {
 
 
   static private MiniDFSCluster dfsCluster;
   static private MiniDFSCluster dfsCluster;
   static private FileSystem fs;
   static private FileSystem fs;
+  static private FileSystem unprivilegedFs;
   static private FileContext fc;
   static private FileContext fc;
+  static private FileContext unprivilegedFc;
   static final private int NUM_OF_PATHS = 4;
   static final private int NUM_OF_PATHS = 4;
   static private String USER_DIR;
   static private String USER_DIR;
   private Path[] path = new Path[NUM_OF_PATHS];
   private Path[] path = new Path[NUM_OF_PATHS];
 
 
   @BeforeClass
   @BeforeClass
   public static void setUp() throws Exception {
   public static void setUp() throws Exception {
-    Configuration conf = new HdfsConfiguration();
+    final Configuration conf = new HdfsConfiguration();
     dfsCluster = new MiniDFSCluster.Builder(conf).build();
     dfsCluster = new MiniDFSCluster.Builder(conf).build();
     fs = FileSystem.get(conf);
     fs = FileSystem.get(conf);
+    unprivilegedFs =
+      unprivilegedUser.doAs(new PrivilegedExceptionAction<FileSystem>() {
+        @Override
+        public FileSystem run() throws IOException {
+          return FileSystem.get(conf);
+        }
+      });
     fc = FileContext.getFileContext(conf);
     fc = FileContext.getFileContext(conf);
+    unprivilegedFc =
+      unprivilegedUser.doAs(new PrivilegedExceptionAction<FileContext>() {
+        @Override
+        public FileContext run() throws IOException {
+          return FileContext.getFileContext(conf);
+        }
+      });
     USER_DIR = fs.getHomeDirectory().toUri().getPath().toString();
     USER_DIR = fs.getHomeDirectory().toUri().getPath().toString();
   }
   }
   
   
@@ -781,8 +803,8 @@ public class TestGlobPaths {
    * A glob test that can be run on either FileContext or FileSystem.
    * A glob test that can be run on either FileContext or FileSystem.
    */
    */
   private static interface FSTestWrapperGlobTest {
   private static interface FSTestWrapperGlobTest {
-    void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception;
+    void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrapper,
+        FileSystem fs, FileContext fc) throws Exception;
   }
   }
 
 
   /**
   /**
@@ -791,7 +813,8 @@ public class TestGlobPaths {
   private void testOnFileSystem(FSTestWrapperGlobTest test) throws Exception {
   private void testOnFileSystem(FSTestWrapperGlobTest test) throws Exception {
     try {
     try {
       fc.mkdir(new Path(USER_DIR), FsPermission.getDefault(), true);
       fc.mkdir(new Path(USER_DIR), FsPermission.getDefault(), true);
-      test.run(new FileSystemTestWrapper(fs), fs, null);
+      test.run(new FileSystemTestWrapper(fs),
+          new FileSystemTestWrapper(unprivilegedFs), fs, null);
     } finally {
     } finally {
       fc.delete(new Path(USER_DIR), true);
       fc.delete(new Path(USER_DIR), true);
     }
     }
@@ -803,7 +826,8 @@ public class TestGlobPaths {
   private void testOnFileContext(FSTestWrapperGlobTest test) throws Exception {
   private void testOnFileContext(FSTestWrapperGlobTest test) throws Exception {
     try {
     try {
       fs.mkdirs(new Path(USER_DIR));
       fs.mkdirs(new Path(USER_DIR));
-      test.run(new FileContextTestWrapper(fc), null, fc);
+      test.run(new FileContextTestWrapper(fc),
+          new FileContextTestWrapper(unprivilegedFc), null, fc);
     } finally {
     } finally {
       cleanupDFS();
       cleanupDFS();
     }
     }
@@ -834,8 +858,8 @@ public class TestGlobPaths {
    * Test globbing through symlinks.
    * Test globbing through symlinks.
    */
    */
   private static class TestGlobWithSymlinks implements FSTestWrapperGlobTest {
   private static class TestGlobWithSymlinks implements FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Test that globbing through a symlink to a directory yields a path
       // Test that globbing through a symlink to a directory yields a path
       // containing that symlink.
       // containing that symlink.
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
@@ -886,8 +910,8 @@ public class TestGlobPaths {
    */
    */
   private static class TestGlobWithSymlinksToSymlinks implements
   private static class TestGlobWithSymlinksToSymlinks implements
       FSTestWrapperGlobTest {
       FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Test that globbing through a symlink to a symlink to a directory
       // Test that globbing through a symlink to a symlink to a directory
       // fully resolves
       // fully resolves
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
@@ -961,8 +985,8 @@ public class TestGlobPaths {
    */
    */
   private static class TestGlobSymlinksWithCustomPathFilter implements
   private static class TestGlobSymlinksWithCustomPathFilter implements
       FSTestWrapperGlobTest {
       FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Test that globbing through a symlink to a symlink to a directory
       // Test that globbing through a symlink to a symlink to a directory
       // fully resolves
       // fully resolves
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
@@ -1009,8 +1033,8 @@ public class TestGlobPaths {
    * Test that globStatus fills in the scheme even when it is not provided.
    * Test that globStatus fills in the scheme even when it is not provided.
    */
    */
   private static class TestGlobFillsInScheme implements FSTestWrapperGlobTest {
   private static class TestGlobFillsInScheme implements FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Verify that the default scheme is hdfs, when we don't supply one.
       // Verify that the default scheme is hdfs, when we don't supply one.
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
           false);
           false);
@@ -1052,8 +1076,8 @@ public class TestGlobPaths {
    * Test that globStatus works with relative paths.
    * Test that globStatus works with relative paths.
    **/
    **/
   private static class TestRelativePath implements FSTestWrapperGlobTest {
   private static class TestRelativePath implements FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-      throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       String[] files = new String[] { "a", "abc", "abc.p", "bacd" };
       String[] files = new String[] { "a", "abc", "abc.p", "bacd" };
 
 
       Path[] path = new Path[files.length];
       Path[] path = new Path[files.length];
@@ -1086,4 +1110,44 @@ public class TestGlobPaths {
   public void testRelativePathOnFC() throws Exception {
   public void testRelativePathOnFC() throws Exception {
     testOnFileContext(new TestRelativePath());
     testOnFileContext(new TestRelativePath());
   }
   }
+  
+  /**
+   * Test that trying to glob through a directory we don't have permission
+   * to list fails with AccessControlException rather than succeeding or
+   * throwing any other exception.
+   **/
+  private static class TestGlobAccessDenied implements FSTestWrapperGlobTest {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
+      wrap.mkdir(new Path("/nopermission/val"),
+          new FsPermission((short)0777), true);
+      wrap.mkdir(new Path("/norestrictions/val"),
+          new FsPermission((short)0777), true);
+      wrap.setPermission(new Path("/nopermission"),
+          new FsPermission((short)0));
+      try {
+        unprivilegedWrap.globStatus(new Path("/no*/*"),
+            new AcceptAllPathFilter());
+        Assert.fail("expected to get an AccessControlException when " +
+            "globbing through a directory we don't have permissions " +
+            "to list.");
+      } catch (AccessControlException ioe) {
+      }
+
+      Assert.assertEquals("/norestrictions/val",
+        TestPath.mergeStatuses(unprivilegedWrap.globStatus(
+            new Path("/norestrictions/*"),
+                new AcceptAllPathFilter())));
+    }
+  }
+
+  @Test
+  public void testGlobAccessDeniedOnFS() throws Exception {
+    testOnFileSystem(new TestGlobAccessDenied());
+  }
+
+  @Test
+  public void testGlobAccessDeniedOnFC() throws Exception {
+    testOnFileContext(new TestGlobAccessDenied());
+  }
 }
 }

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

@@ -190,6 +190,17 @@ Release 2.2.0 - UNRELEASED
 
 
   BUG FIXES
   BUG FIXES
 
 
+    MAPREDUCE-5504. mapred queue -info inconsistent with types (Kousuke Saruta
+    via tgraves)
+
+    MAPREDUCE-5488. Changed MR client to keep trying to reach the application
+    when it sees that on attempt's AM is down. (Jian He via vinodkv)
+
+    MAPREDUCE-5515. Fixed MR AM's webapp to depend on a new config
+    mapreduce.ssl.enabled to enable https and disabling it by default as MR AM
+    needs to set up its own certificates etc and not depend on clusters'.
+    (Omkar Vinit Joshi via vinodkv)
+
 Release 2.1.1-beta - 2013-09-23
 Release 2.1.1-beta - 2013-09-23
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -1365,6 +1376,9 @@ Release 0.23.10 - UNRELEASED
 
 
     MAPREDUCE-5475. MRClientService does not verify ACLs properly (jlowe)
     MAPREDUCE-5475. MRClientService does not verify ACLs properly (jlowe)
 
 
+    MAPREDUCE-5504. mapred queue -info inconsistent with types (Kousuke Saruta
+    via tgraves)
+
 Release 0.23.9 - 2013-07-08
 Release 0.23.9 - 2013-07-08
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 6 - 0
hadoop-mapreduce-project/dev-support/findbugs-exclude.xml

@@ -496,6 +496,12 @@
       <Field name="sslFileBufferSize" />
       <Field name="sslFileBufferSize" />
      <Bug pattern="IS2_INCONSISTENT_SYNC" />
      <Bug pattern="IS2_INCONSISTENT_SYNC" />
    </Match> 
    </Match> 
+
+   <Match>
+     <Class name="org.apache.hadoop.mapred.ClientServiceDelegate" />
+      <Method name="invoke" />
+     <Bug pattern="SWL_SLEEP_WITH_LOCK_HELD" />
+   </Match>
    
    
   <Match>
   <Match>
     <Class name="org.apache.hadoop.mapreduce.util.ProcessTree" />
     <Class name="org.apache.hadoop.mapreduce.util.ProcessTree" />

+ 15 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -36,14 +36,17 @@ import org.apache.commons.io.IOUtils;
 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.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.mapred.FileOutputCommitter;
 import org.apache.hadoop.mapred.FileOutputCommitter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.LocalContainerLauncher;
 import org.apache.hadoop.mapred.LocalContainerLauncher;
 import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
 import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.OutputFormat;
@@ -101,6 +104,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
@@ -1313,6 +1317,7 @@ public class MRAppMaster extends CompositeService {
           containerId.getApplicationAttemptId();
           containerId.getApplicationAttemptId();
       long appSubmitTime = Long.parseLong(appSubmitTimeStr);
       long appSubmitTime = Long.parseLong(appSubmitTimeStr);
       
       
+      
       MRAppMaster appMaster =
       MRAppMaster appMaster =
           new MRAppMaster(applicationAttemptId, containerId, nodeHostString,
           new MRAppMaster(applicationAttemptId, containerId, nodeHostString,
               Integer.parseInt(nodePortString),
               Integer.parseInt(nodePortString),
@@ -1322,6 +1327,16 @@ public class MRAppMaster extends CompositeService {
         new MRAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY);
         new MRAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY);
       JobConf conf = new JobConf(new YarnConfiguration());
       JobConf conf = new JobConf(new YarnConfiguration());
       conf.addResource(new Path(MRJobConfig.JOB_CONF_FILE));
       conf.addResource(new Path(MRJobConfig.JOB_CONF_FILE));
+      
+      // Explicitly disabling SSL for map reduce task as we can't allow MR users
+      // to gain access to keystore file for opening SSL listener. We can trust
+      // RM/NM to issue SSL certificates but definitely not MR-AM as it is
+      // running in user-land.
+      HttpConfig.setSecure(conf.getBoolean(MRConfig.SSL_ENABLED_KEY,
+          MRConfig.SSL_ENABLED_KEY_DEFAULT));
+      WebAppUtil.setSSLEnabledInYARN(conf.getBoolean(
+          CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_KEY,
+          CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_DEFAULT));
 
 
       // log the system properties
       // log the system properties
       String systemPropsToLog = MRApps.getSystemPropertiesToLog(conf);
       String systemPropsToLog = MRApps.getSystemPropertiesToLog(conf);

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

@@ -27,8 +27,10 @@ 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.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
+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.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
@@ -78,6 +80,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;

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

@@ -28,7 +28,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 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.http.HttpConfig;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobID;
+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.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -36,12 +38,10 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 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.event.JobEvent;
-import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@@ -58,8 +58,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
 
-import com.sun.research.ws.wadl.Response;
-
 /**
 /**
  * Registers/unregisters to RM and sends heartbeats to RM.
  * Registers/unregisters to RM and sends heartbeats to RM.
  */
  */
@@ -148,7 +146,13 @@ public abstract class RMCommunicator extends AbstractService
       if (serviceAddr != null) {
       if (serviceAddr != null) {
         request.setHost(serviceAddr.getHostName());
         request.setHost(serviceAddr.getHostName());
         request.setRpcPort(serviceAddr.getPort());
         request.setRpcPort(serviceAddr.getPort());
-        request.setTrackingUrl(serviceAddr.getHostName() + ":" + clientService.getHttpPort());
+        String scheme = "http://";
+        if (getConfig().getBoolean(MRConfig.SSL_ENABLED_KEY,
+            MRConfig.SSL_ENABLED_KEY_DEFAULT)) {
+          scheme = "https://";
+        }
+        request.setTrackingUrl(scheme + serviceAddr.getHostName() + ":"
+            + clientService.getHttpPort());
       }
       }
       RegisterApplicationMasterResponse response =
       RegisterApplicationMasterResponse response =
         scheduler.registerApplicationMaster(request);
         scheduler.registerApplicationMaster(request);
@@ -190,10 +194,11 @@ public abstract class RMCommunicator extends AbstractService
       }
       }
       LOG.info("Setting job diagnostics to " + sb.toString());
       LOG.info("Setting job diagnostics to " + sb.toString());
 
 
-      String historyUrl = JobHistoryUtils.getHistoryUrl(getConfig(),
-          context.getApplicationID());
+      String historyUrl =
+          WebAppUtil.getSchemePrefix()
+              + JobHistoryUtils.getHistoryUrl(getConfig(),
+                  context.getApplicationID());
       LOG.info("History url is " + historyUrl);
       LOG.info("History url is " + historyUrl);
-
       FinishApplicationMasterRequest request =
       FinishApplicationMasterRequest request =
           FinishApplicationMasterRequest.newInstance(finishState,
           FinishApplicationMasterRequest.newInstance(finishState,
             sb.toString(), historyUrl);
             sb.toString(), historyUrl);

+ 5 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.Controller;
 import org.apache.hadoop.yarn.webapp.Controller;
 import org.apache.hadoop.yarn.webapp.View;
 import org.apache.hadoop.yarn.webapp.View;
 
 
+import com.google.common.base.Joiner;
 import com.google.inject.Inject;
 import com.google.inject.Inject;
 
 
 /**
 /**
@@ -50,6 +51,7 @@ import com.google.inject.Inject;
  */
  */
 public class AppController extends Controller implements AMParams {
 public class AppController extends Controller implements AMParams {
   private static final Log LOG = LogFactory.getLog(AppController.class);
   private static final Log LOG = LogFactory.getLog(AppController.class);
+  private static final Joiner JOINER = Joiner.on("");
   
   
   protected final App app;
   protected final App app;
   
   
@@ -58,7 +60,9 @@ public class AppController extends Controller implements AMParams {
     super(ctx);
     super(ctx);
     this.app = app;
     this.app = app;
     set(APP_ID, app.context.getApplicationID().toString());
     set(APP_ID, app.context.getApplicationID().toString());
-    set(RM_WEB, YarnConfiguration.getRMWebAppURL(conf));
+    set(RM_WEB,
+        JOINER.join(WebAppUtil.getSchemePrefix(),
+            YarnConfiguration.getRMWebAppHostAndPort(conf)));
   }
   }
 
 
   @Inject
   @Inject

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

@@ -104,7 +104,7 @@ public class JobBlock extends HtmlBlock {
       table.tr().
       table.tr().
         td(String.valueOf(attempt.getAttemptId())).
         td(String.valueOf(attempt.getAttemptId())).
         td(new Date(attempt.getStartTime()).toString()).
         td(new Date(attempt.getStartTime()).toString()).
-        td().a(".nodelink", url(HttpConfig.getSchemePrefix(),
+        td().a(".nodelink", url(WebAppUtil.getSchemePrefix(),
             attempt.getNodeHttpAddress()),
             attempt.getNodeHttpAddress()),
             attempt.getNodeHttpAddress())._().
             attempt.getNodeHttpAddress())._().
         td().a(".logslink", url(attempt.getLogsLink()), 
         td().a(".logslink", url(attempt.getLogsLink()), 

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

@@ -63,7 +63,7 @@ public class NavBlock extends HtmlBlock {
           li().a(url("conf", jobid), "Configuration")._().
           li().a(url("conf", jobid), "Configuration")._().
           li().a(url("tasks", jobid, "m"), "Map tasks")._().
           li().a(url("tasks", jobid, "m"), "Map tasks")._().
           li().a(url("tasks", jobid, "r"), "Reduce tasks")._().
           li().a(url("tasks", jobid, "r"), "Reduce tasks")._().
-          li().a(".logslink", url(HttpConfig.getSchemePrefix(),
+          li().a(".logslink", url(WebAppUtil.getSchemePrefix(),
               nodeHttpAddress, "node",
               nodeHttpAddress, "node",
               "containerlogs", thisAmInfo.getContainerId().toString(), 
               "containerlogs", thisAmInfo.getContainerId().toString(), 
               app.getJob().getUserName()), 
               app.getJob().getUserName()), 

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

@@ -86,12 +86,12 @@ public class TaskPage extends AppView {
         .append(ta.getState().toString()).append("\",\"")
         .append(ta.getState().toString()).append("\",\"")
 
 
         .append(nodeHttpAddr == null ? "N/A" :
         .append(nodeHttpAddr == null ? "N/A" :
-          "<a class='nodelink' href='" + HttpConfig.getSchemePrefix() + nodeHttpAddr + "'>"
+          "<a class='nodelink' href='" + WebAppUtil.getSchemePrefix() + nodeHttpAddr + "'>"
           + nodeHttpAddr + "</a>")
           + nodeHttpAddr + "</a>")
         .append("\",\"")
         .append("\",\"")
 
 
         .append(ta.getAssignedContainerId() == null ? "N/A" :
         .append(ta.getAssignedContainerId() == null ? "N/A" :
-          "<a class='logslink' href='" + url(HttpConfig.getSchemePrefix(), nodeHttpAddr, "node"
+          "<a class='logslink' href='" + url(WebAppUtil.getSchemePrefix(), nodeHttpAddr, "node"
             , "containerlogs", ta.getAssignedContainerIdStr(), app.getJob()
             , "containerlogs", ta.getAssignedContainerIdStr(), app.getJob()
             .getUserName()) + "'>logs</a>")
             .getUserName()) + "'>logs</a>")
           .append("\",\"")
           .append("\",\"")

+ 39 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/WebAppUtil.java

@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.v2.app.webapp;
+
+
+public class WebAppUtil {
+  private static boolean isSSLEnabledInYARN;
+  
+  public static void setSSLEnabledInYARN(boolean isSSLEnabledInYARN) {
+    WebAppUtil.isSSLEnabledInYARN = isSSLEnabledInYARN;
+  }
+  
+  public static boolean isSSLEnabledInYARN() {
+    return isSSLEnabledInYARN;
+  }
+  
+  public static String getSchemePrefix() {
+    if (isSSLEnabledInYARN) {
+      return "https://";
+    } else {
+      return "http://";
+    }
+  }
+}

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

@@ -26,6 +26,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 
 
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 
 
@@ -63,7 +64,7 @@ public class AMAttemptInfo {
     ContainerId containerId = amInfo.getContainerId();
     ContainerId containerId = amInfo.getContainerId();
     if (containerId != null) {
     if (containerId != null) {
       this.containerId = containerId.toString();
       this.containerId = containerId.toString();
-      this.logsLink = join(HttpConfig.getSchemePrefix() + nodeHttpAddress,
+      this.logsLink = join(WebAppUtil.getSchemePrefix() + nodeHttpAddress,
           ujoin("node", "containerlogs", this.containerId, user));
           ujoin("node", "containerlogs", this.containerId, user));
     }
     }
   }
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -469,7 +469,7 @@ public class TypeConverter {
     QueueInfo toReturn = new QueueInfo(queueInfo.getQueueName(), "Capacity: " +
     QueueInfo toReturn = new QueueInfo(queueInfo.getQueueName(), "Capacity: " +
       queueInfo.getCapacity() * 100 + ", MaximumCapacity: " +
       queueInfo.getCapacity() * 100 + ", MaximumCapacity: " +
       (queueInfo.getMaximumCapacity() < 0 ? "UNDEFINED" :
       (queueInfo.getMaximumCapacity() < 0 ? "UNDEFINED" :
-        queueInfo.getMaximumCapacity()) + ", CurrentCapacity: " +
+        queueInfo.getMaximumCapacity() * 100) + ", CurrentCapacity: " +
       queueInfo.getCurrentCapacity() * 100, fromYarn(queueInfo.getQueueState()),
       queueInfo.getCurrentCapacity() * 100, fromYarn(queueInfo.getQueueState()),
       TypeConverter.fromYarnApps(queueInfo.getApplications(), conf));
       TypeConverter.fromYarnApps(queueInfo.getApplications(), conf));
     List<QueueInfo> childQueues = new ArrayList<QueueInfo>();
     List<QueueInfo> childQueues = new ArrayList<QueueInfo>();

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRConfig.java

@@ -84,6 +84,11 @@ public interface MRConfig {
     "mapreduce.shuffle.ssl.enabled";
     "mapreduce.shuffle.ssl.enabled";
 
 
   public static final boolean SHUFFLE_SSL_ENABLED_DEFAULT = false;
   public static final boolean SHUFFLE_SSL_ENABLED_DEFAULT = false;
+  
+  public static final String SSL_ENABLED_KEY =
+      "mapreduce.ssl.enabled";
+  
+  public static final boolean SSL_ENABLED_KEY_DEFAULT = false;
 
 
   public static final String SHUFFLE_CONSUMER_PLUGIN =
   public static final String SHUFFLE_CONSUMER_PLUGIN =
     "mapreduce.job.reduce.shuffle.consumer.plugin.class";
     "mapreduce.job.reduce.shuffle.consumer.plugin.class";

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

@@ -357,7 +357,7 @@ public interface MRJobConfig {
   public static final int DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 3;
   public static final int DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 3;
   
   
   /**
   /**
-   * The number of client retries to the RM/HS/AM before throwing exception.
+   * The number of client retries to the RM/HS before throwing exception.
    */
    */
   public static final String MR_CLIENT_MAX_RETRIES = 
   public static final String MR_CLIENT_MAX_RETRIES = 
     MR_PREFIX + "client.max-retries";
     MR_PREFIX + "client.max-retries";

+ 16 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -289,6 +289,20 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>mapreduce.ssl.enabled</name>
+  <value>false</value>
+  <description>
+	If enabled, MapReduce application master's http server will be
+	started with SSL enabled. Map reduce AM by default doesn't support SSL.
+	If MapReduce jobs want SSL support, it is the user's responsibility to
+	create and manage certificates, keystores and trust-stores with appropriate
+	permissions. This is only for MapReduce application master and is not used
+	by job history server. To enable encrypted shuffle this property is not
+	required, instead refer to (mapreduce.shuffle.ssl.enabled) property.
+  </description>
+</property>
+
 <property>
 <property>
   <name>mapreduce.shuffle.ssl.file.buffer.size</name>
   <name>mapreduce.shuffle.ssl.file.buffer.size</name>
   <value>65536</value>
   <value>65536</value>
@@ -982,7 +996,7 @@
 
 
 <property>
 <property>
   <name>yarn.app.mapreduce.client-am.ipc.max-retries</name>
   <name>yarn.app.mapreduce.client-am.ipc.max-retries</name>
-  <value>1</value>
+  <value>3</value>
   <description>The number of client retries to the AM - before reconnecting
   <description>The number of client retries to the AM - before reconnecting
     to the RM to fetch Application Status.</description>
     to the RM to fetch Application Status.</description>
 </property>
 </property>
@@ -990,7 +1004,7 @@
 <property>
 <property>
   <name>yarn.app.mapreduce.client.max-retries</name>
   <name>yarn.app.mapreduce.client.max-retries</name>
   <value>3</value>
   <value>3</value>
-  <description>The number of client retries to the RM/HS/AM before
+  <description>The number of client retries to the RM/HS before
     throwing exception. This is a layer above the ipc.</description>
     throwing exception. This is a layer above the ipc.</description>
 </property>
 </property>
 
 

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

@@ -24,8 +24,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.mapreduce.v2.hs.server.HSAdminServer;
 import org.apache.hadoop.mapreduce.v2.hs.server.HSAdminServer;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -73,6 +75,10 @@ public class JobHistoryServer extends CompositeService {
 
 
     config.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
     config.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
 
 
+    // This is required for WebApps to use https if enabled.
+    WebAppUtil.setSSLEnabledInYARN(conf.getBoolean(
+        CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_KEY,
+        CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_DEFAULT));
     try {
     try {
       doSecureLogin(conf);
       doSecureLogin(conf);
     } catch(IOException ie) {
     } catch(IOException ie) {

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
@@ -132,7 +133,7 @@ public class HsJobBlock extends HtmlBlock {
             table.tr((odd = !odd) ? _ODD : _EVEN).
             table.tr((odd = !odd) ? _ODD : _EVEN).
               td(String.valueOf(attempt.getAttemptId())).
               td(String.valueOf(attempt.getAttemptId())).
               td(new Date(attempt.getStartTime()).toString()).
               td(new Date(attempt.getStartTime()).toString()).
-              td().a(".nodelink", url(HttpConfig.getSchemePrefix(),
+              td().a(".nodelink", url(WebAppUtil.getSchemePrefix(),
                   attempt.getNodeHttpAddress()),
                   attempt.getNodeHttpAddress()),
                   attempt.getNodeHttpAddress())._().
                   attempt.getNodeHttpAddress())._().
               td().a(".logslink", url(attempt.getShortLogsLink()), 
               td().a(".logslink", url(attempt.getShortLogsLink()), 

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 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.webapp.App;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.mapreduce.v2.app.webapp.WebAppUtil;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.Times;
@@ -148,7 +149,7 @@ public class HsTaskPage extends HsView {
         .append(sortId + " ").append(taid).append("\",\"")
         .append(sortId + " ").append(taid).append("\",\"")
         .append(ta.getState().toString()).append("\",\"")
         .append(ta.getState().toString()).append("\",\"")
 
 
-        .append("<a class='nodelink' href='" + HttpConfig.getSchemePrefix() + nodeHttpAddr + "'>")
+        .append("<a class='nodelink' href='" + WebAppUtil.getSchemePrefix() + nodeHttpAddr + "'>")
         .append(nodeRackName + "/" + nodeHttpAddr + "</a>\",\"")
         .append(nodeRackName + "/" + nodeHttpAddr + "</a>\",\"")
 
 
         .append("<a class='logslink' href='").append(url("logs", nodeIdString
         .append("<a class='logslink' href='").append(url("logs", nodeIdString

+ 37 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -26,6 +26,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -77,6 +78,8 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class ClientServiceDelegate {
 public class ClientServiceDelegate {
   private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class);
   private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class);
   private static final String UNAVAILABLE = "N/A";
   private static final String UNAVAILABLE = "N/A";
@@ -93,7 +96,8 @@ public class ClientServiceDelegate {
   private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private static String UNKNOWN_USER = "Unknown User";
   private static String UNKNOWN_USER = "Unknown User";
   private String trackingUrl;
   private String trackingUrl;
-
+  private AtomicBoolean usingAMProxy = new AtomicBoolean(false);
+  private int maxClientRetry;
   private boolean amAclDisabledStatusLogged = false;
   private boolean amAclDisabledStatusLogged = false;
 
 
   public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
   public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
@@ -287,6 +291,7 @@ public class ClientServiceDelegate {
     MRClientProtocol proxy = 
     MRClientProtocol proxy = 
          (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
          (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
             serviceAddr, conf);
             serviceAddr, conf);
+    usingAMProxy.set(true);
     LOG.trace("Connected to ApplicationMaster at: " + serviceAddr);
     LOG.trace("Connected to ApplicationMaster at: " + serviceAddr);
     return proxy;
     return proxy;
   }
   }
@@ -301,13 +306,15 @@ public class ClientServiceDelegate {
     } catch (NoSuchMethodException e) {
     } catch (NoSuchMethodException e) {
       throw new YarnRuntimeException("Method name mismatch", e);
       throw new YarnRuntimeException("Method name mismatch", e);
     }
     }
-    int maxRetries = this.conf.getInt(
+    maxClientRetry = this.conf.getInt(
         MRJobConfig.MR_CLIENT_MAX_RETRIES,
         MRJobConfig.MR_CLIENT_MAX_RETRIES,
         MRJobConfig.DEFAULT_MR_CLIENT_MAX_RETRIES);
         MRJobConfig.DEFAULT_MR_CLIENT_MAX_RETRIES);
     IOException lastException = null;
     IOException lastException = null;
-    while (maxRetries > 0) {
+    while (maxClientRetry > 0) {
+      MRClientProtocol MRClientProxy = null;
       try {
       try {
-        return methodOb.invoke(getProxy(), args);
+        MRClientProxy = getProxy();
+        return methodOb.invoke(MRClientProxy, args);
       } catch (InvocationTargetException e) {
       } catch (InvocationTargetException e) {
         // Will not throw out YarnException anymore
         // Will not throw out YarnException anymore
         LOG.debug("Failed to contact AM/History for job " + jobId + 
         LOG.debug("Failed to contact AM/History for job " + jobId + 
@@ -315,22 +322,44 @@ public class ClientServiceDelegate {
         // Force reconnection by setting the proxy to null.
         // Force reconnection by setting the proxy to null.
         realProxy = null;
         realProxy = null;
         // HS/AMS shut down
         // HS/AMS shut down
-        maxRetries--;
+        // if it's AM shut down, do not decrement maxClientRetry as we wait for
+        // AM to be restarted.
+        if (!usingAMProxy.get()) {
+          maxClientRetry--;
+        }
+        usingAMProxy.set(false);
         lastException = new IOException(e.getTargetException());
         lastException = new IOException(e.getTargetException());
-        
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException ie) {
+          LOG.warn("ClientServiceDelegate invoke call interrupted", ie);
+          throw new YarnRuntimeException(ie);
+        }
       } catch (Exception e) {
       } catch (Exception e) {
         LOG.debug("Failed to contact AM/History for job " + jobId
         LOG.debug("Failed to contact AM/History for job " + jobId
             + "  Will retry..", e);
             + "  Will retry..", e);
         // Force reconnection by setting the proxy to null.
         // Force reconnection by setting the proxy to null.
         realProxy = null;
         realProxy = null;
         // RM shutdown
         // RM shutdown
-        maxRetries--;
-        lastException = new IOException(e.getMessage());     
+        maxClientRetry--;
+        lastException = new IOException(e.getMessage());
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException ie) {
+          LOG.warn("ClientServiceDelegate invoke call interrupted", ie);
+          throw new YarnRuntimeException(ie);
+        }
       }
       }
     }
     }
     throw lastException;
     throw lastException;
   }
   }
 
 
+  // Only for testing
+  @VisibleForTesting
+  public int getMaxClientRetry() {
+    return this.maxClientRetry;
+  }
+
   public org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException,
   public org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException,
   InterruptedException {
   InterruptedException {
     org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0);
     org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0);

+ 42 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java

@@ -140,6 +140,48 @@ public class TestClientServiceDelegate {
         any(GetJobReportRequest.class));
         any(GetJobReportRequest.class));
   }
   }
 
 
+  @Test
+  public void testRetriesOnAMConnectionFailures() throws Exception {
+    if (!isAMReachableFromClient) {
+      return;
+    }
+
+    ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
+    when(rm.getApplicationReport(TypeConverter.toYarn(oldJobId).getAppId()))
+      .thenReturn(getRunningApplicationReport("am1", 78));
+
+    // throw exception in 1st, 2nd, 3rd and 4th call of getJobReport, and
+    // succeed in the 5th call.
+    final MRClientProtocol amProxy = mock(MRClientProtocol.class);
+    when(amProxy.getJobReport(any(GetJobReportRequest.class)))
+      .thenThrow(new RuntimeException("11"))
+      .thenThrow(new RuntimeException("22"))
+      .thenThrow(new RuntimeException("33"))
+      .thenThrow(new RuntimeException("44")).thenReturn(getJobReportResponse());
+    Configuration conf = new YarnConfiguration();
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
+    conf.setBoolean(MRJobConfig.JOB_AM_ACCESS_DISABLED,
+      !isAMReachableFromClient);
+    ClientServiceDelegate clientServiceDelegate =
+        new ClientServiceDelegate(conf, rm, oldJobId, null) {
+          @Override
+          MRClientProtocol instantiateAMProxy(
+              final InetSocketAddress serviceAddr) throws IOException {
+            super.instantiateAMProxy(serviceAddr);
+            return amProxy;
+          }
+        };
+
+    JobStatus jobStatus = clientServiceDelegate.getJobStatus(oldJobId);
+
+    Assert.assertNotNull(jobStatus);
+    // assert maxClientRetry is not decremented.
+    Assert.assertEquals(conf.getInt(MRJobConfig.MR_CLIENT_MAX_RETRIES,
+      MRJobConfig.DEFAULT_MR_CLIENT_MAX_RETRIES), clientServiceDelegate
+      .getMaxClientRetry());
+    verify(amProxy, times(5)).getJobReport(any(GetJobReportRequest.class));
+  }
+
   @Test
   @Test
   public void testHistoryServerNotConfigured() throws Exception {
   public void testHistoryServerNotConfigured() throws Exception {
     //RM doesn't have app report and job History Server is not configured
     //RM doesn't have app report and job History Server is not configured

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

@@ -27,10 +27,15 @@ Release 2.3.0 - UNRELEASED
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     YARN-905. Add state filters to nodes CLI (Wei Yan via Sandy Ryza)
     YARN-905. Add state filters to nodes CLI (Wei Yan via Sandy Ryza)
+
     YARN-1098. Separate out RM services into Always On and Active (Karthik
     YARN-1098. Separate out RM services into Always On and Active (Karthik
     Kambatla via bikas)
     Kambatla via bikas)
+
     YARN-1027. Implement RMHAProtocolService (Karthik Kambatla via bikas)
     YARN-1027. Implement RMHAProtocolService (Karthik Kambatla via bikas)
 
 
+    YARN-353. Add Zookeeper-based store implementation for RMStateStore.
+    (Bikas Saha, Jian He and Karthik Kambatla via hitesh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -50,6 +55,9 @@ Release 2.2.0 - UNRELEASED
 
 
   BUG FIXES
   BUG FIXES
 
 
+    YARN-1128. FifoPolicy.computeShares throws NPE on empty list of Schedulables
+    (Karthik Kambatla via Sandy Ryza)
+
 Release 2.1.1-beta - 2013-09-23
 Release 2.1.1-beta - 2013-09-23
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -121,6 +129,9 @@ Release 2.1.1-beta - 2013-09-23
     YARN-1001. Added a web-service to get statistics about per application-type
     YARN-1001. Added a web-service to get statistics about per application-type
     per state for consumption by downstream projects. (Zhijie Shen via vinodkv)
     per state for consumption by downstream projects. (Zhijie Shen via vinodkv)
 
 
+    YARN-1203. Changed YARN web-app proxy to handle http and https URLs from
+    AM registration and finish correctly. (Omkar Vinit Joshi via vinodkv)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java

@@ -91,6 +91,8 @@ public abstract class FinishApplicationMasterRequest {
 
 
   /**
   /**
    * Get the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
    * Get the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
+   * This url if contains scheme then that will be used by resource manager
+   * web application proxy otherwise it will default to http.
    * @return <em>tracking URL</em>for the <code>ApplicationMaster</code>
    * @return <em>tracking URL</em>for the <code>ApplicationMaster</code>
    */
    */
   @Public
   @Public
@@ -99,6 +101,8 @@ public abstract class FinishApplicationMasterRequest {
 
 
   /**
   /**
    * Set the <em>tracking URL</em>for the <code>ApplicationMaster</code>
    * Set the <em>tracking URL</em>for the <code>ApplicationMaster</code>
+   * This url if contains scheme then that will be used by resource manager
+   * web application proxy otherwise it will default to http.
    * @param url <em>tracking URL</em>for the
    * @param url <em>tracking URL</em>for the
    *                   <code>ApplicationMaster</code>
    *                   <code>ApplicationMaster</code>
    */
    */

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java

@@ -103,6 +103,8 @@ public abstract class RegisterApplicationMasterRequest {
 
 
   /**
   /**
    * Get the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
    * Get the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
+   * This url if contains scheme then that will be used by resource manager
+   * web application proxy otherwise it will default to http.
    * @return <em>tracking URL</em> for the <code>ApplicationMaster</code>
    * @return <em>tracking URL</em> for the <code>ApplicationMaster</code>
    */
    */
   @Public
   @Public
@@ -111,6 +113,8 @@ public abstract class RegisterApplicationMasterRequest {
   
   
   /**
   /**
    * Set the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
    * Set the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
+   * This url if contains scheme then that will be used by resource manager
+   * web application proxy otherwise it will default to http.
    * @param trackingUrl <em>tracking URL</em> for the 
    * @param trackingUrl <em>tracking URL</em> for the 
    *                    <code>ApplicationMaster</code>
    *                    <code>ApplicationMaster</code>
    */
    */

+ 32 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -26,6 +26,8 @@ import java.util.Arrays;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -276,12 +278,40 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_HA_ENABLED = RM_HA_PREFIX + "enabled";
   public static final String RM_HA_ENABLED = RM_HA_PREFIX + "enabled";
   public static final boolean DEFAULT_RM_HA_ENABLED = false;
   public static final boolean DEFAULT_RM_HA_ENABLED = false;
   
   
+
+  ////////////////////////////////
+  // RM state store configs
+  ////////////////////////////////
   /** The class to use as the persistent store.*/
   /** The class to use as the persistent store.*/
   public static final String RM_STORE = RM_PREFIX + "store.class";
   public static final String RM_STORE = RM_PREFIX + "store.class";
   
   
   /** URI for FileSystemRMStateStore */
   /** URI for FileSystemRMStateStore */
   public static final String FS_RM_STATE_STORE_URI =
   public static final String FS_RM_STATE_STORE_URI =
                                            RM_PREFIX + "fs.state-store.uri";
                                            RM_PREFIX + "fs.state-store.uri";
+  /**
+   * Comma separated host:port pairs, each corresponding to a ZK server for
+   * ZKRMStateStore
+   */
+  public static final String ZK_STATE_STORE_PREFIX =
+      RM_PREFIX + "zk.state-store.";
+  public static final String ZK_RM_STATE_STORE_NUM_RETRIES =
+      ZK_STATE_STORE_PREFIX + "num-retries";
+  public static final int DEFAULT_ZK_RM_STATE_STORE_NUM_RETRIES = 3;
+  public static final String ZK_RM_STATE_STORE_ADDRESS =
+      ZK_STATE_STORE_PREFIX + "address";
+  /** Timeout in millisec for ZK server connection for ZKRMStateStore */
+  public static final String ZK_RM_STATE_STORE_TIMEOUT_MS =
+      ZK_STATE_STORE_PREFIX + "timeout.ms";
+  public static final int DEFAULT_ZK_RM_STATE_STORE_TIMEOUT_MS = 60000;
+  /** Parent znode path under which ZKRMStateStore will create znodes */
+  public static final String ZK_RM_STATE_STORE_PARENT_PATH =
+      ZK_STATE_STORE_PREFIX + "parent-path";
+  public static final String DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH = "/rmstore";
+  /** ACL for znodes in ZKRMStateStore */
+  public static final String ZK_RM_STATE_STORE_ACL =
+      ZK_STATE_STORE_PREFIX + "acl";
+  public static final String DEFAULT_ZK_RM_STATE_STORE_ACL =
+      "world:anyone:rwcda";
 
 
   /** The maximum number of completed applications RM keeps. */ 
   /** The maximum number of completed applications RM keeps. */ 
   public static final String RM_MAX_COMPLETED_APPLICATIONS =
   public static final String RM_MAX_COMPLETED_APPLICATIONS =
@@ -837,7 +867,8 @@ public class YarnConfiguration extends Configuration {
   }
   }
   
   
   public static String getRMWebAppURL(Configuration conf) {
   public static String getRMWebAppURL(Configuration conf) {
-    return JOINER.join("http://", getRMWebAppHostAndPort(conf));
+    return JOINER.join(HttpConfig.getSchemePrefix(),
+        getRMWebAppHostAndPort(conf));
   }
   }
   
   
 }
 }

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -258,6 +258,51 @@
     <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
     <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
   </property>
   </property>
 
 
+  <property>
+    <description>Host:Port of the ZooKeeper server where RM state will 
+    be stored. This must be supplied when using
+    org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
+    as the value for yarn.resourcemanager.store.class</description>
+    <name>yarn.resourcemanager.zk.state-store.address</name>
+    <!--value>127.0.0.1:2181</value-->
+  </property>
+
+  <property>
+    <description>Number of times ZKRMStateStore tries to connect to
+    ZooKeeper. This may be supplied when using
+    org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
+    as the value for yarn.resourcemanager.store.class</description>
+    <name>yarn.resourcemanager.zk.state-store.num-retries</name>
+    <value>3</value>
+  </property>
+
+  <property>
+    <description>Full path of the ZooKeeper znode where RM state will be
+    stored. This must be supplied when using
+    org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
+    as the value for yarn.resourcemanager.store.class</description>
+    <name>yarn.resourcemanager.zk.state-store.parent-path</name>
+    <value>/rmstore</value>
+  </property>
+
+  <property>
+    <description>Timeout when connecting to ZooKeeper.
+    This may be supplied when using
+    org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
+    as the value for yarn.resourcemanager.store.class</description>
+    <name>yarn.resourcemanager.zk.state-store.timeout.ms</name>
+    <value>60000</value>
+  </property>
+
+  <property>
+    <description>ACL's to be used for ZooKeeper znodes.
+    This may be supplied when using
+    org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
+    as the value for yarn.resourcemanager.store.class</description>
+    <name>yarn.resourcemanager.zk.state-store.acl</name>
+    <value>world:anyone:rwcda</value>
+  </property>
+
   <property>
   <property>
     <description>URI pointing to the location of the FileSystem path where
     <description>URI pointing to the location of the FileSystem path where
     RM state will be stored. This must be supplied when using
     RM state will be stored. This must be supplied when using

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml

@@ -41,6 +41,16 @@
       <groupId>org.apache.hadoop</groupId>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-web-proxy</artifactId>
       <artifactId>hadoop-yarn-server-web-proxy</artifactId>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
     <dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <artifactId>hadoop-hdfs</artifactId>

+ 0 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java

@@ -63,12 +63,6 @@ public class FileSystemRMStateStore extends RMStateStore {
   public static final Log LOG = LogFactory.getLog(FileSystemRMStateStore.class);
   public static final Log LOG = LogFactory.getLog(FileSystemRMStateStore.class);
 
 
   private static final String ROOT_DIR_NAME = "FSRMStateRoot";
   private static final String ROOT_DIR_NAME = "FSRMStateRoot";
-  private static final String RM_DT_SECRET_MANAGER_ROOT = "RMDTSecretManagerRoot";
-  private static final String RM_APP_ROOT = "RMAppRoot";
-  private static final String DELEGATION_KEY_PREFIX = "DelegationKey_";
-  private static final String DELEGATION_TOKEN_PREFIX = "RMDelegationToken_";
-  private static final String DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX =
-      "RMDTSequenceNumber_";
 
 
   protected FileSystem fs;
   protected FileSystem fs;
 
 

+ 11 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java

@@ -65,6 +65,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
  */
  */
 public abstract class RMStateStore extends AbstractService {
 public abstract class RMStateStore extends AbstractService {
 
 
+  // constants for RM App state and RMDTSecretManagerState.
+  protected static final String RM_APP_ROOT = "RMAppRoot";
+  protected static final String RM_DT_SECRET_MANAGER_ROOT = "RMDTSecretManagerRoot";
+  protected static final String DELEGATION_KEY_PREFIX = "DelegationKey_";
+  protected static final String DELEGATION_TOKEN_PREFIX = "RMDelegationToken_";
+  protected static final String DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX =
+      "RMDTSequenceNumber_";
+
   public static final Log LOG = LogFactory.getLog(RMStateStore.class);
   public static final Log LOG = LogFactory.getLog(RMStateStore.class);
 
 
   public RMStateStore() {
   public RMStateStore() {
@@ -464,8 +472,9 @@ public abstract class RMStateStore extends AbstractService {
               (ApplicationAttemptStateDataPBImpl) ApplicationAttemptStateDataPBImpl
               (ApplicationAttemptStateDataPBImpl) ApplicationAttemptStateDataPBImpl
                   .newApplicationAttemptStateData(attemptState.getAttemptId(),
                   .newApplicationAttemptStateData(attemptState.getAttemptId(),
                     attemptState.getMasterContainer(), appAttemptTokens);
                     attemptState.getMasterContainer(), appAttemptTokens);
-
-            LOG.info("Storing info for attempt: " + attemptState.getAttemptId());
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Storing info for attempt: " + attemptState.getAttemptId());
+            }
             storeApplicationAttemptState(attemptState.getAttemptId().toString(), 
             storeApplicationAttemptState(attemptState.getAttemptId().toString(), 
                                          attemptStateData);
                                          attemptStateData);
           } catch (Exception e) {
           } catch (Exception e) {

+ 621 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java

@@ -0,0 +1,621 @@
+/**
+ * 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.yarn.server.resourcemanager.recovery;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.delegation.DelegationKey;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
+import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ZKUtil;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+@Private
+@Unstable
+public class ZKRMStateStore extends RMStateStore {
+
+  public static final Log LOG = LogFactory.getLog(ZKRMStateStore.class);
+
+  private static final String ROOT_ZNODE_NAME = "ZKRMStateRoot";
+  private int numRetries;
+
+  private String zkHostPort = null;
+  private int zkSessionTimeout;
+  private List<ACL> zkAcl;
+  private String zkRootNodePath;
+  private String rmDTSecretManagerRoot;
+  private String rmAppRoot;
+  private String dtSequenceNumberPath = null;
+
+  @VisibleForTesting
+  protected String znodeWorkingPath;
+
+  @VisibleForTesting
+  protected ZooKeeper zkClient;
+  private ZooKeeper oldZkClient;
+
+  @Override
+  public synchronized void initInternal(Configuration conf) throws Exception {
+    zkHostPort = conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS);
+    if (zkHostPort == null) {
+      throw new YarnRuntimeException("No server address specified for " +
+          "zookeeper state store for Resource Manager recovery. " +
+          YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS + " is not configured.");
+    }
+    numRetries =
+        conf.getInt(YarnConfiguration.ZK_RM_STATE_STORE_NUM_RETRIES,
+            YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_NUM_RETRIES);
+    znodeWorkingPath =
+        conf.get(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH,
+            YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH);
+    zkSessionTimeout =
+        conf.getInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS,
+            YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_TIMEOUT_MS);
+    // Parse authentication from configuration.
+    String zkAclConf =
+        conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ACL,
+            YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_ACL);
+    zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
+
+    try {
+      zkAcl = ZKUtil.parseACLs(zkAclConf);
+    } catch (ZKUtil.BadAclFormatException bafe) {
+      LOG.error("Invalid format for " + YarnConfiguration.ZK_RM_STATE_STORE_ACL);
+      throw bafe;
+    }
+
+    zkRootNodePath = znodeWorkingPath + "/" + ROOT_ZNODE_NAME;
+    rmDTSecretManagerRoot = zkRootNodePath + "/" + RM_DT_SECRET_MANAGER_ROOT;
+    rmAppRoot = zkRootNodePath + "/" + RM_APP_ROOT;
+  }
+
+  @Override
+  public synchronized void startInternal() throws Exception {
+    // createConnection for future API calls
+    createConnection();
+
+    // ensure root dirs exist
+    createRootDir(znodeWorkingPath);
+    createRootDir(zkRootNodePath);
+    createRootDir(rmDTSecretManagerRoot);
+    createRootDir(rmAppRoot);
+  }
+
+  private void createRootDir(String rootPath) throws Exception {
+    try {
+      createWithRetries(rootPath, null, zkAcl, CreateMode.PERSISTENT);
+    } catch (KeeperException ke) {
+      if (ke.code() != Code.NODEEXISTS) {
+        throw ke;
+      }
+    }
+  }
+
+  private synchronized void closeZkClients() throws IOException {
+    if (zkClient != null) {
+      try {
+        zkClient.close();
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while closing ZK", e);
+      }
+      zkClient = null;
+    }
+    if (oldZkClient != null) {
+      try {
+        oldZkClient.close();
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while closing old ZK", e);
+      }
+      oldZkClient = null;
+    }
+  }
+
+  @Override
+  protected synchronized void closeInternal() throws Exception {
+    closeZkClients();
+  }
+
+  @Override
+  public synchronized RMState loadState() throws Exception {
+    RMState rmState = new RMState();
+    // recover DelegationTokenSecretManager
+    loadRMDTSecretManagerState(rmState);
+    // recover RM applications
+    loadRMAppState(rmState);
+    return rmState;
+  }
+
+  private synchronized void loadRMDTSecretManagerState(RMState rmState)
+      throws Exception {
+    List<String> childNodes = zkClient.getChildren(rmDTSecretManagerRoot, true);
+
+    for (String childNodeName : childNodes) {
+      if (childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
+        rmState.rmSecretManagerState.dtSequenceNumber =
+            Integer.parseInt(childNodeName.split("_")[1]);
+        continue;
+      }
+      String childNodePath = getNodePath(rmDTSecretManagerRoot, childNodeName);
+      byte[] childData = getDataWithRetries(childNodePath, true);
+
+      ByteArrayInputStream is = new ByteArrayInputStream(childData);
+      DataInputStream fsIn = new DataInputStream(is);
+      try {
+        if (childNodeName.startsWith(DELEGATION_KEY_PREFIX)) {
+          DelegationKey key = new DelegationKey();
+          key.readFields(fsIn);
+          rmState.rmSecretManagerState.masterKeyState.add(key);
+        } else if (childNodeName.startsWith(DELEGATION_TOKEN_PREFIX)) {
+          RMDelegationTokenIdentifier identifier =
+              new RMDelegationTokenIdentifier();
+          identifier.readFields(fsIn);
+          long renewDate = fsIn.readLong();
+          rmState.rmSecretManagerState.delegationTokenState.put(identifier,
+              renewDate);
+        }
+      } finally {
+        is.close();
+      }
+    }
+  }
+
+  private synchronized void loadRMAppState(RMState rmState) throws Exception {
+    List<String> childNodes = zkClient.getChildren(rmAppRoot, true);
+    List<ApplicationAttemptState> attempts =
+        new ArrayList<ApplicationAttemptState>();
+    for (String childNodeName : childNodes) {
+      String childNodePath = getNodePath(rmAppRoot, childNodeName);
+      byte[] childData = getDataWithRetries(childNodePath, true);
+      if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
+        // application
+        LOG.info("Loading application from znode: " + childNodeName);
+        ApplicationId appId = ConverterUtils.toApplicationId(childNodeName);
+        ApplicationStateDataPBImpl appStateData =
+            new ApplicationStateDataPBImpl(
+                ApplicationStateDataProto.parseFrom(childData));
+        ApplicationState appState =
+            new ApplicationState(appStateData.getSubmitTime(),
+                appStateData.getApplicationSubmissionContext(),
+                appStateData.getUser());
+        if (!appId.equals(appState.context.getApplicationId())) {
+          throw new YarnRuntimeException("The child node name is different " +
+              "from the application id");
+        }
+        rmState.appState.put(appId, appState);
+      } else if (childNodeName
+          .startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
+        // attempt
+        LOG.info("Loading application attempt from znode: " + childNodeName);
+        ApplicationAttemptId attemptId =
+            ConverterUtils.toApplicationAttemptId(childNodeName);
+        ApplicationAttemptStateDataPBImpl attemptStateData =
+            new ApplicationAttemptStateDataPBImpl(
+                ApplicationAttemptStateDataProto.parseFrom(childData));
+        Credentials credentials = null;
+        if (attemptStateData.getAppAttemptTokens() != null) {
+          credentials = new Credentials();
+          DataInputByteBuffer dibb = new DataInputByteBuffer();
+          dibb.reset(attemptStateData.getAppAttemptTokens());
+          credentials.readTokenStorageStream(dibb);
+        }
+        ApplicationAttemptState attemptState =
+            new ApplicationAttemptState(attemptId,
+                attemptStateData.getMasterContainer(), credentials);
+        if (!attemptId.equals(attemptState.getAttemptId())) {
+          throw new YarnRuntimeException("The child node name is different " +
+              "from the application attempt id");
+        }
+        attempts.add(attemptState);
+      } else {
+        LOG.info("Unknown child node with name: " + childNodeName);
+      }
+    }
+
+    // go through all attempts and add them to their apps
+    for (ApplicationAttemptState attemptState : attempts) {
+      ApplicationId appId = attemptState.getAttemptId().getApplicationId();
+      ApplicationState appState = rmState.appState.get(appId);
+      if (appState != null) {
+        appState.attempts.put(attemptState.getAttemptId(), attemptState);
+      } else {
+        // the application znode may have been removed when the application
+        // completed but the RM might have stopped before it could remove the
+        // application attempt znodes
+        LOG.info("Application node not found for attempt: "
+            + attemptState.getAttemptId());
+        deleteWithRetries(
+            getNodePath(rmAppRoot, attemptState.getAttemptId().toString()),
+            0);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void storeApplicationState(
+      String appId, ApplicationStateDataPBImpl appStateDataPB) throws
+      Exception {
+    String nodeCreatePath = getNodePath(rmAppRoot, appId);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Storing info for app: " + appId + " at: " + nodeCreatePath);
+    }
+    byte[] appStateData = appStateDataPB.getProto().toByteArray();
+    createWithRetries(
+        nodeCreatePath, appStateData, zkAcl, CreateMode.PERSISTENT);
+  }
+
+  @Override
+  public synchronized void storeApplicationAttemptState(
+      String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
+      throws Exception {
+    String nodeCreatePath = getNodePath(rmAppRoot, attemptId);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Storing info for attempt: " + attemptId + " at: "
+          + nodeCreatePath);
+    }
+    byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
+    createWithRetries(nodeCreatePath, attemptStateData, zkAcl,
+        CreateMode.PERSISTENT);
+  }
+
+  @Override
+  public synchronized void removeApplicationState(ApplicationState appState)
+      throws Exception {
+    String appId = appState.getAppId().toString();
+    String nodeRemovePath = getNodePath(rmAppRoot, appId);
+    ArrayList<Op> opList = new ArrayList<Op>();
+    opList.add(Op.delete(nodeRemovePath, 0));
+
+    for (ApplicationAttemptId attemptId : appState.attempts.keySet()) {
+      String attemptRemovePath = getNodePath(rmAppRoot, attemptId.toString());
+      opList.add(Op.delete(attemptRemovePath, 0));
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing info for app: " + appId + " at: " + nodeRemovePath
+          + " and its attempts.");
+    }
+    doMultiWithRetries(opList);
+  }
+
+  @Override
+  protected synchronized void storeRMDelegationTokenAndSequenceNumberState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
+      int latestSequenceNumber) throws Exception {
+    ArrayList<Op> opList = new ArrayList<Op>();
+    // store RM delegation token
+    String nodeCreatePath =
+        getNodePath(rmDTSecretManagerRoot, DELEGATION_TOKEN_PREFIX
+            + rmDTIdentifier.getSequenceNumber());
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    DataOutputStream fsOut = new DataOutputStream(os);
+    try {
+      rmDTIdentifier.write(fsOut);
+      fsOut.writeLong(renewDate);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Storing RMDelegationToken_" +
+            rmDTIdentifier.getSequenceNumber());
+      }
+      opList.add(Op.create(nodeCreatePath, os.toByteArray(), zkAcl,
+          CreateMode.PERSISTENT));
+    } finally {
+      os.close();
+    }
+
+    // store sequence number
+    String latestSequenceNumberPath =
+        getNodePath(rmDTSecretManagerRoot,
+            DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX + latestSequenceNumber);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Storing " + DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX +
+          latestSequenceNumber);
+    }
+
+    if (dtSequenceNumberPath != null) {
+      opList.add(Op.delete(dtSequenceNumberPath, 0));
+    }
+    opList.add(Op.create(latestSequenceNumberPath, null, zkAcl,
+        CreateMode.PERSISTENT));
+    dtSequenceNumberPath = latestSequenceNumberPath;
+    doMultiWithRetries(opList);
+  }
+
+  @Override
+  protected synchronized void removeRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier) throws Exception {
+    String nodeRemovePath =
+        getNodePath(rmDTSecretManagerRoot, DELEGATION_TOKEN_PREFIX
+            + rmDTIdentifier.getSequenceNumber());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing RMDelegationToken_"
+          + rmDTIdentifier.getSequenceNumber());
+    }
+    deleteWithRetries(nodeRemovePath, 0);
+  }
+
+  @Override
+  protected synchronized void storeRMDTMasterKeyState(
+      DelegationKey delegationKey) throws Exception {
+    String nodeCreatePath =
+        getNodePath(rmDTSecretManagerRoot, DELEGATION_KEY_PREFIX
+            + delegationKey.getKeyId());
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    DataOutputStream fsOut = new DataOutputStream(os);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Storing RMDelegationKey_" + delegationKey.getKeyId());
+    }
+    delegationKey.write(fsOut);
+    try {
+      createWithRetries(nodeCreatePath, os.toByteArray(), zkAcl,
+          CreateMode.PERSISTENT);
+    } finally {
+      os.close();
+    }
+  }
+
+  @Override
+  protected synchronized void removeRMDTMasterKeyState(
+      DelegationKey delegationKey) throws Exception {
+    String nodeRemovePath =
+        getNodePath(rmDTSecretManagerRoot, DELEGATION_KEY_PREFIX
+            + delegationKey.getKeyId());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing RMDelegationKey_" + delegationKey.getKeyId());
+    }
+    deleteWithRetries(nodeRemovePath, 0);
+  }
+
+  // ZK related code
+  /**
+   * Watcher implementation which forward events to the ZKRMStateStore This
+   * hides the ZK methods of the store from its public interface
+   */
+  private final class ForwardingWatcher implements Watcher {
+
+    @Override
+    public void process(WatchedEvent event) {
+      try {
+        ZKRMStateStore.this.processWatchEvent(event);
+      } catch (Throwable t) {
+        LOG.error("Failed to process watcher event " + event + ": "
+            + StringUtils.stringifyException(t));
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public synchronized void processWatchEvent(WatchedEvent event)
+      throws Exception {
+    Event.EventType eventType = event.getType();
+    LOG.info("Watcher event type: " + eventType + " with state:"
+        + event.getState() + " for path:" + event.getPath() + " for " + this);
+
+    if (eventType == Event.EventType.None) {
+
+      // the connection state has changed
+      switch (event.getState()) {
+        case SyncConnected:
+          LOG.info("ZKRMStateStore Session connected");
+          if (oldZkClient != null) {
+            // the SyncConnected must be from the client that sent Disconnected
+            zkClient = oldZkClient;
+            oldZkClient = null;
+            ZKRMStateStore.this.notifyAll();
+            LOG.info("ZKRMStateStore Session restored");
+          }
+          break;
+        case Disconnected:
+          LOG.info("ZKRMStateStore Session disconnected");
+          oldZkClient = zkClient;
+          zkClient = null;
+          break;
+        case Expired:
+          // the connection got terminated because of session timeout
+          // call listener to reconnect
+          LOG.info("Session expired");
+          createConnection();
+          break;
+        default:
+          LOG.error("Unexpected Zookeeper" +
+              " watch event state: " + event.getState());
+          break;
+      }
+    }
+  }
+
+  @VisibleForTesting
+  String getNodePath(String root, String nodeName) {
+    return (root + "/" + nodeName);
+  }
+
+  @VisibleForTesting
+  public String createWithRetries(
+      final String path, final byte[] data, final List<ACL> acl,
+      final CreateMode mode) throws Exception {
+    return new ZKAction<String>() {
+      @Override
+      public String run() throws KeeperException, InterruptedException {
+        return zkClient.create(path, data, acl, mode);
+      }
+    }.runWithRetries();
+  }
+
+  private void deleteWithRetries(final String path, final int version)
+      throws Exception {
+    new ZKAction<Void>() {
+      @Override
+      public Void run() throws KeeperException, InterruptedException {
+        /**
+         * Call exists() to leave a watch on the node denoted by path.
+         * Delete node if exists. To pass the existence information to the
+         * caller, call delete irrespective of whether node exists or not.
+         */
+        if (zkClient.exists(path, true) == null) {
+          LOG.error("Trying to delete a path (" + path
+              + ") that doesn't exist.");
+        }
+        zkClient.delete(path, version);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  private void doMultiWithRetries(final ArrayList<Op> opList) throws Exception {
+    new ZKAction<Void>() {
+      @Override
+      public Void run() throws KeeperException, InterruptedException {
+        zkClient.multi(opList);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  @VisibleForTesting
+  public void setDataWithRetries(final String path, final byte[] data,
+                                 final int version) throws Exception {
+    new ZKAction<Void>() {
+      @Override
+      public Void run() throws KeeperException, InterruptedException {
+        zkClient.setData(path, data, version);
+        return null;
+      }
+    }.runWithRetries();
+  }
+
+  @VisibleForTesting
+  public byte[] getDataWithRetries(final String path, final boolean watch)
+      throws Exception {
+    return new ZKAction<byte[]>() {
+      @Override
+      public byte[] run() throws KeeperException, InterruptedException {
+        Stat stat = new Stat();
+        return zkClient.getData(path, watch, stat);
+      }
+    }.runWithRetries();
+  }
+
+  private abstract class ZKAction<T> {
+    // run() expects synchronization on ZKRMStateStore.this
+    abstract T run() throws KeeperException, InterruptedException;
+
+    T runWithCheck() throws Exception {
+      long startTime = System.currentTimeMillis();
+      synchronized (ZKRMStateStore.this) {
+        while (zkClient == null) {
+          ZKRMStateStore.this.wait(zkSessionTimeout);
+          if (zkClient != null) {
+            break;
+          }
+          if (System.currentTimeMillis() - startTime > zkSessionTimeout) {
+            throw new IOException("Wait for ZKClient creation timed out");
+          }
+        }
+        return run();
+      }
+    }
+
+    T runWithRetries() throws Exception {
+      int retry = 0;
+      while (true) {
+        try {
+          return runWithCheck();
+        } catch (KeeperException ke) {
+          if (shouldRetry(ke.code()) && ++retry < numRetries) {
+            continue;
+          }
+          throw ke;
+        }
+      }
+    }
+  }
+
+  private static boolean shouldRetry(Code code) {
+    switch (code) {
+      case CONNECTIONLOSS:
+      case OPERATIONTIMEOUT:
+        return true;
+      default:
+        break;
+    }
+    return false;
+  }
+
+  private synchronized void createConnection()
+      throws IOException, InterruptedException {
+    closeZkClients();
+    for (int retries = 0; retries < numRetries && zkClient == null;
+        retries++) {
+      try {
+        zkClient = getNewZooKeeper();
+      } catch (IOException ioe) {
+        // Retry in case of network failures
+        LOG.info("Failed to connect to the ZooKeeper on attempt - " +
+            (retries + 1));
+        ioe.printStackTrace();
+      }
+    }
+    if (zkClient == null) {
+      LOG.error("Unable to connect to Zookeeper");
+      throw new YarnRuntimeException("Unable to connect to Zookeeper");
+    }
+    ZKRMStateStore.this.notifyAll();
+    LOG.info("Created new ZK connection");
+  }
+
+  // protected to mock for testing
+  @VisibleForTesting
+  protected synchronized ZooKeeper getNewZooKeeper()
+      throws IOException, InterruptedException {
+    ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, null);
+    zk.register(new ForwardingWatcher());
+    return zk;
+  }
+}

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies;
 import java.io.Serializable;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Comparator;
+import java.util.Iterator;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -73,6 +74,10 @@ public class FifoPolicy extends SchedulingPolicy {
   @Override
   @Override
   public void computeShares(Collection<? extends Schedulable> schedulables,
   public void computeShares(Collection<? extends Schedulable> schedulables,
       Resource totalResources) {
       Resource totalResources) {
+    if (schedulables.isEmpty()) {
+      return;
+    }
+
     Schedulable earliest = null;
     Schedulable earliest = null;
     for (Schedulable schedulable : schedulables) {
     for (Schedulable schedulable : schedulables) {
       if (earliest == null ||
       if (earliest == null ||

+ 70 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStore.java

@@ -26,8 +26,10 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
 import javax.crypto.SecretKey;
 import javax.crypto.SecretKey;
@@ -40,6 +42,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -67,13 +70,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+
+import org.apache.zookeeper.ZooKeeper;
+
 import org.junit.Test;
 import org.junit.Test;
 
 
-public class TestRMStateStore {
+public class TestRMStateStore extends ClientBaseWithFixes{
 
 
   public static final Log LOG = LogFactory.getLog(TestRMStateStore.class);
   public static final Log LOG = LogFactory.getLog(TestRMStateStore.class);
 
 
-  class TestDispatcher implements Dispatcher, EventHandler<RMAppAttemptStoredEvent> {
+  static class TestDispatcher implements
+      Dispatcher, EventHandler<RMAppAttemptStoredEvent> {
 
 
     ApplicationAttemptId attemptId;
     ApplicationAttemptId attemptId;
     Exception storedException;
     Exception storedException;
@@ -82,7 +89,8 @@ public class TestRMStateStore {
 
 
     @SuppressWarnings("rawtypes")
     @SuppressWarnings("rawtypes")
     @Override
     @Override
-    public void register(Class<? extends Enum> eventType, EventHandler handler) {
+    public void register(Class<? extends Enum> eventType,
+                         EventHandler handler) {
     }
     }
 
 
     @Override
     @Override
@@ -108,10 +116,18 @@ public class TestRMStateStore {
     boolean isFinalStateValid() throws Exception;
     boolean isFinalStateValid() throws Exception;
   }
   }
 
 
+  @Test
+  public void testZKRMStateStoreRealZK() throws Exception {
+    TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
+    testRMAppStateStore(zkTester);
+    testRMDTSecretManagerStateStore(zkTester);
+  }
+
   @Test
   @Test
   public void testFSRMStateStore() throws Exception {
   public void testFSRMStateStore() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     HdfsConfiguration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     try {
     try {
       TestFSRMStateStoreTester fsTester = new TestFSRMStateStoreTester(cluster);
       TestFSRMStateStoreTester fsTester = new TestFSRMStateStoreTester(cluster);
       testRMAppStateStore(fsTester);
       testRMAppStateStore(fsTester);
@@ -121,6 +137,41 @@ public class TestRMStateStore {
     }
     }
   }
   }
 
 
+  class TestZKRMStateStoreTester implements RMStateStoreHelper {
+    ZooKeeper client;
+    ZKRMStateStore store;
+
+    class TestZKRMStateStore extends ZKRMStateStore {
+      public TestZKRMStateStore(Configuration conf, String workingZnode)
+          throws Exception {
+        init(conf);
+        start();
+        assertTrue(znodeWorkingPath.equals(workingZnode));
+      }
+
+      @Override
+      public ZooKeeper getNewZooKeeper() throws IOException {
+        return client;
+      }
+    }
+
+    public RMStateStore getRMStateStore() throws Exception {
+      String workingZnode = "/Test";
+      YarnConfiguration conf = new YarnConfiguration();
+      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
+      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
+      this.client = createClient();
+      this.store = new TestZKRMStateStore(conf, workingZnode);
+      return this.store;
+    }
+
+    @Override
+    public boolean isFinalStateValid() throws Exception {
+      List<String> nodes = client.getChildren(store.znodeWorkingPath, false);
+      return nodes.size() == 1;
+    }
+  }
+
   class TestFSRMStateStoreTester implements RMStateStoreHelper {
   class TestFSRMStateStoreTester implements RMStateStoreHelper {
     Path workingDirPathURI;
     Path workingDirPathURI;
     FileSystemRMStateStore store;
     FileSystemRMStateStore store;
@@ -149,7 +200,8 @@ public class TestRMStateStore {
     @Override
     @Override
     public RMStateStore getRMStateStore() throws Exception {
     public RMStateStore getRMStateStore() throws Exception {
       YarnConfiguration conf = new YarnConfiguration();
       YarnConfiguration conf = new YarnConfiguration();
-      conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI, workingDirPathURI.toString());
+      conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
+          workingDirPathURI.toString());
       this.store = new TestFileSystemRMStore(conf);
       this.store = new TestFileSystemRMStore(conf);
       return store;
       return store;
     }
     }
@@ -158,11 +210,7 @@ public class TestRMStateStore {
     public boolean isFinalStateValid() throws Exception {
     public boolean isFinalStateValid() throws Exception {
       FileSystem fs = cluster.getFileSystem();
       FileSystem fs = cluster.getFileSystem();
       FileStatus[] files = fs.listStatus(workingDirPathURI);
       FileStatus[] files = fs.listStatus(workingDirPathURI);
-      if(files.length == 1) {
-        // only store root directory should exist
-        return true;
-      }
-      return false;
+      return files.length == 1;
     }
     }
   }
   }
 
 
@@ -183,9 +231,10 @@ public class TestRMStateStore {
     dispatcher.notified = false;
     dispatcher.notified = false;
   }
   }
 
 
-  void storeApp(RMStateStore store, ApplicationId appId, long time)
-                                                              throws Exception {
-    ApplicationSubmissionContext context = new ApplicationSubmissionContextPBImpl();
+  void storeApp(
+      RMStateStore store, ApplicationId appId, long time) throws Exception {
+    ApplicationSubmissionContext context =
+        new ApplicationSubmissionContextPBImpl();
     context.setApplicationId(appId);
     context.setApplicationId(appId);
 
 
     RMApp mockApp = mock(RMApp.class);
     RMApp mockApp = mock(RMApp.class);
@@ -216,7 +265,8 @@ public class TestRMStateStore {
     return container.getId();
     return container.getId();
   }
   }
 
 
-  void testRMAppStateStore(RMStateStoreHelper stateStoreHelper) throws Exception {
+  void testRMAppStateStore(RMStateStoreHelper stateStoreHelper)
+      throws Exception {
     long submitTime = System.currentTimeMillis();
     long submitTime = System.currentTimeMillis();
     Configuration conf = new YarnConfiguration();
     Configuration conf = new YarnConfiguration();
     RMStateStore store = stateStoreHelper.getRMStateStore();
     RMStateStore store = stateStoreHelper.getRMStateStore();
@@ -271,7 +321,8 @@ public class TestRMStateStore {
     RMApp mockRemovedApp = mock(RMApp.class);
     RMApp mockRemovedApp = mock(RMApp.class);
     HashMap<ApplicationAttemptId, RMAppAttempt> attempts =
     HashMap<ApplicationAttemptId, RMAppAttempt> attempts =
                               new HashMap<ApplicationAttemptId, RMAppAttempt>();
                               new HashMap<ApplicationAttemptId, RMAppAttempt>();
-    ApplicationSubmissionContext context = new ApplicationSubmissionContextPBImpl();
+    ApplicationSubmissionContext context =
+        new ApplicationSubmissionContextPBImpl();
     context.setApplicationId(appIdRemoved);
     context.setApplicationId(appIdRemoved);
     when(mockRemovedApp.getSubmitTime()).thenReturn(submitTime);
     when(mockRemovedApp.getSubmitTime()).thenReturn(submitTime);
     when(mockRemovedApp.getApplicationSubmissionContext()).thenReturn(context);
     when(mockRemovedApp.getApplicationSubmissionContext()).thenReturn(context);
@@ -288,7 +339,8 @@ public class TestRMStateStore {
     // load state
     // load state
     store = stateStoreHelper.getRMStateStore();
     store = stateStoreHelper.getRMStateStore();
     RMState state = store.loadState();
     RMState state = store.loadState();
-    Map<ApplicationId, ApplicationState> rmAppState = state.getApplicationState();
+    Map<ApplicationId, ApplicationState> rmAppState =
+        state.getApplicationState();
 
 
     ApplicationState appState = rmAppState.get(appId1);
     ApplicationState appState = rmAppState.get(appId1);
     // app is loaded
     // app is loaded
@@ -362,7 +414,8 @@ public class TestRMStateStore {
         store.loadState().getRMDTSecretManagerState();
         store.loadState().getRMDTSecretManagerState();
     Assert.assertEquals(token1, secretManagerState.getTokenState());
     Assert.assertEquals(token1, secretManagerState.getTokenState());
     Assert.assertEquals(keySet, secretManagerState.getMasterKeyState());
     Assert.assertEquals(keySet, secretManagerState.getMasterKeyState());
-    Assert.assertEquals(sequenceNumber, secretManagerState.getDTSequenceNumber());
+    Assert.assertEquals(sequenceNumber,
+        secretManagerState.getDTSequenceNumber());
   }
   }
 
 
   private Token<AMRMTokenIdentifier> generateAMRMToken(
   private Token<AMRMTokenIdentifier> generateAMRMToken(

+ 218 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStoreZKClientConnections.java

@@ -0,0 +1,218 @@
+/**
+ * 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.yarn.server.resourcemanager.recovery;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.ClientBaseWithFixes;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.TestRMStateStore.TestDispatcher;
+import org.apache.hadoop.util.ZKUtil;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.CyclicBarrier;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestZKRMStateStoreZKClientConnections extends
+    ClientBaseWithFixes {
+  private static final int ZK_OP_WAIT_TIME = 3000;
+  private Log LOG =
+      LogFactory.getLog(TestZKRMStateStoreZKClientConnections.class);
+
+  class TestZKClient {
+    ZKRMStateStore store;
+    boolean forExpire = false;
+    TestForwardingWatcher watcher;
+    CyclicBarrier syncBarrier = new CyclicBarrier(2);
+
+    protected class TestZKRMStateStore extends ZKRMStateStore {
+      public TestZKRMStateStore(Configuration conf, String workingZnode)
+          throws Exception {
+        init(conf);
+        start();
+        assertTrue(znodeWorkingPath.equals(workingZnode));
+      }
+
+      @Override
+      public ZooKeeper getNewZooKeeper()
+          throws IOException, InterruptedException {
+        return createClient(watcher, hostPort, 100);
+      }
+
+      @Override
+      public synchronized void processWatchEvent(WatchedEvent event)
+          throws Exception {
+
+        if (forExpire) {
+          // a hack... couldn't find a way to trigger expired event.
+          WatchedEvent expriredEvent = new WatchedEvent(
+              Watcher.Event.EventType.None,
+              Watcher.Event.KeeperState.Expired, null);
+          super.processWatchEvent(expriredEvent);
+          forExpire = false;
+          syncBarrier.await();
+        } else {
+          super.processWatchEvent(event);
+        }
+      }
+    }
+
+    private class TestForwardingWatcher extends
+        ClientBaseWithFixes.CountdownWatcher {
+      public void process(WatchedEvent event) {
+        super.process(event);
+        try {
+          if (store != null) {
+            store.processWatchEvent(event);
+          }
+        } catch (Throwable t) {
+          LOG.error("Failed to process watcher event " + event + ": "
+              + StringUtils.stringifyException(t));
+        }
+      }
+    }
+
+    public RMStateStore getRMStateStore(Configuration conf) throws Exception {
+      String workingZnode = "/Test";
+      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
+      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
+      watcher = new TestForwardingWatcher();
+      this.store = new TestZKRMStateStore(conf, workingZnode);
+      return this.store;
+    }
+  }
+
+  @Test(timeout = 20000)
+  public void testZKClientDisconnectAndReconnect()
+      throws Exception {
+
+    TestZKClient zkClientTester = new TestZKClient();
+    String path = "/test";
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS, 100);
+    ZKRMStateStore store =
+        (ZKRMStateStore) zkClientTester.getRMStateStore(conf);
+    TestDispatcher dispatcher = new TestDispatcher();
+    store.setRMDispatcher(dispatcher);
+
+    // trigger watch
+    store.createWithRetries(path, null, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+        CreateMode.PERSISTENT);
+    store.getDataWithRetries(path, true);
+    store.setDataWithRetries(path, "newBytes".getBytes(), 0);
+
+    stopServer();
+    zkClientTester.watcher.waitForDisconnected(ZK_OP_WAIT_TIME);
+    try {
+      store.getDataWithRetries(path, true);
+      fail("Expected ZKClient time out exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains(
+          "Wait for ZKClient creation timed out"));
+    }
+
+    // ZKRMStateStore Session restored
+    startServer();
+    zkClientTester.watcher.waitForConnected(ZK_OP_WAIT_TIME);
+    byte[] ret = null;
+    try {
+      ret = store.getDataWithRetries(path, true);
+    } catch (Exception e) {
+      String error = "ZKRMStateStore Session restore failed";
+      LOG.error(error, e);
+      fail(error);
+    }
+    Assert.assertEquals("newBytes", new String(ret));
+  }
+
+  @Test(timeout = 20000)
+  public void testZKSessionTimeout() throws Exception {
+
+    TestZKClient zkClientTester = new TestZKClient();
+    String path = "/test";
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS, 100);
+    ZKRMStateStore store =
+        (ZKRMStateStore) zkClientTester.getRMStateStore(conf);
+    TestDispatcher dispatcher = new TestDispatcher();
+    store.setRMDispatcher(dispatcher);
+
+    // a hack to trigger expired event
+    zkClientTester.forExpire = true;
+
+    // trigger watch
+    store.createWithRetries(path, null, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+        CreateMode.PERSISTENT);
+    store.getDataWithRetries(path, true);
+    store.setDataWithRetries(path, "bytes".getBytes(), 0);
+
+    zkClientTester.syncBarrier.await();
+    // after this point, expired event has already been processed.
+
+    try {
+      byte[] ret = store.getDataWithRetries(path, false);
+      Assert.assertEquals("bytes", new String(ret));
+    } catch (Exception e) {
+      String error = "New session creation failed";
+      LOG.error(error, e);
+      fail(error);
+    }
+  }
+
+  @Test (timeout = 20000)
+  public void testSetZKAcl() {
+    TestZKClient zkClientTester = new TestZKClient();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ACL, "world:anyone:rwca");
+    try {
+      zkClientTester.store.zkClient.delete(zkClientTester.store
+          .znodeWorkingPath, -1);
+      fail("Shouldn't be able to delete path");
+    } catch (Exception e) {/* expected behavior */}
+  }
+
+  @Test (timeout = 20000)
+  public void testInvalidZKAclConfiguration() {
+    TestZKClient zkClientTester = new TestZKClient();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ACL, "randomstring&*");
+    try {
+      zkClientTester.getRMStateStore(conf);
+      fail("ZKRMStateStore created with bad ACL");
+    } catch (ZKUtil.BadAclFormatException bafe) {
+      // expected behavior
+    } catch (Exception e) {
+      String error = "Incorrect exception on BadAclFormat";
+      LOG.error(error, e);
+      fail(error);
+    }
+  }
+}

+ 57 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestEmptyQueues.java

@@ -0,0 +1,57 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.fair.policies;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+public class TestEmptyQueues {
+  private Collection<? extends Schedulable> schedulables;
+
+  @Before
+  public void setup() {
+    schedulables = new ArrayList<Schedulable>();
+  }
+
+  private void testComputeShares(SchedulingPolicy policy) {
+    policy.computeShares(schedulables, Resources.none());
+  }
+
+  @Test (timeout = 1000)
+  public void testFifoPolicy() {
+    testComputeShares(SchedulingPolicy.getInstance(FifoPolicy.class));
+  }
+
+  @Test (timeout = 1000)
+  public void testFairSharePolicy() {
+    testComputeShares(SchedulingPolicy.getInstance(FairSharePolicy.class));
+  }
+
+  @Test (timeout = 1000)
+  public void testDRFPolicy() {
+    testComputeShares(
+        SchedulingPolicy.getInstance(DominantResourceFairnessPolicy.class));
+  }
+}

+ 49 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java

@@ -137,14 +137,44 @@ public class ProxyUriUtils {
   
   
   /**
   /**
    * Create a URI form a no scheme Url, such as is returned by the AM.
    * Create a URI form a no scheme Url, such as is returned by the AM.
-   * @param noSchemeUrl the URL formate returned by an AM
+   * @param url the URL format returned by an AM. This may or may not contain
+   * scheme.
    * @return a URI with an http scheme
    * @return a URI with an http scheme
    * @throws URISyntaxException if the url is not formatted correctly.
    * @throws URISyntaxException if the url is not formatted correctly.
    */
    */
-  public static URI getUriFromAMUrl(String noSchemeUrl)
+  public static URI getUriFromAMUrl(String url)
     throws URISyntaxException {
     throws URISyntaxException {
-      return new URI(HttpConfig.getSchemePrefix() + noSchemeUrl);
+    if (getSchemeFromUrl(url).isEmpty()) {
+      /*
+       * check is made to make sure if AM reports with scheme then it will be
+       * used by default otherwise it will default to the one configured using
+       * "hadoop.ssl.enabled".
+       */
+      return new URI(HttpConfig.getSchemePrefix() + url);      
+    } else {
+      return new URI(url);
+    }
   }
   }
+  
+  /**
+   * Create a URI form a no scheme Url, such as is returned by the AM.
+   * @param noSchemeUrl the URL formate returned by an AM
+   * @return a URI with an http scheme
+   * @throws URISyntaxException if the url is not formatted correctly.
+   */
+  public static URI getUriFromAMUrl(String scheme, String noSchemeUrl)
+      throws URISyntaxException {
+      if (getSchemeFromUrl(noSchemeUrl).isEmpty()) {
+        /*
+         * check is made to make sure if AM reports with scheme then it will be
+         * used by default otherwise it will default to the one configured using
+         * "hadoop.ssl.enabled".
+         */
+        return new URI(scheme + "://" + noSchemeUrl);      
+      } else {
+        return new URI(noSchemeUrl);
+      }
+    }
 
 
   /**
   /**
    * Returns the first valid tracking link, if any, from the given id from the
    * Returns the first valid tracking link, if any, from the given id from the
@@ -169,4 +199,20 @@ public class ProxyUriUtils {
     }
     }
     return null;
     return null;
   }
   }
+  
+  /**
+   * Returns the scheme if present in the url
+   * eg. "https://issues.apache.org/jira/browse/YARN" > "https"
+   */
+  public static String getSchemeFromUrl(String url) {
+    int index = 0;
+    if (url != null) {
+      index = url.indexOf("://");
+    }
+    if (index > 0) {
+      return url.substring(0, index);
+    } else {
+      return "";
+    }
+  }
 }
 }

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java

@@ -163,7 +163,6 @@ public class WebAppProxyServlet extends HttpServlet {
     }
     }
     config.setLocalAddress(localAddress);
     config.setLocalAddress(localAddress);
     HttpMethod method = new GetMethod(uri.getEscapedURI());
     HttpMethod method = new GetMethod(uri.getEscapedURI());
-
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")
     Enumeration<String> names = req.getHeaderNames();
     Enumeration<String> names = req.getHeaderNames();
     while(names.hasMoreElements()) {
     while(names.hasMoreElements()) {
@@ -293,14 +292,17 @@ public class WebAppProxyServlet extends HttpServlet {
       }
       }
       String original = applicationReport.getOriginalTrackingUrl();
       String original = applicationReport.getOriginalTrackingUrl();
       URI trackingUri = null;
       URI trackingUri = null;
-      if (original != null) {
-        trackingUri = ProxyUriUtils.getUriFromAMUrl(original);
-      }
       // fallback to ResourceManager's app page if no tracking URI provided
       // fallback to ResourceManager's app page if no tracking URI provided
       if(original == null || original.equals("N/A")) {
       if(original == null || original.equals("N/A")) {
         resp.sendRedirect(resp.encodeRedirectURL(
         resp.sendRedirect(resp.encodeRedirectURL(
             StringHelper.pjoin(rmAppPageUrlBase, id.toString())));
             StringHelper.pjoin(rmAppPageUrlBase, id.toString())));
         return;
         return;
+      } else {
+        if (ProxyUriUtils.getSchemeFromUrl(original).isEmpty()) {
+          trackingUri = ProxyUriUtils.getUriFromAMUrl("http", original);
+        } else {
+          trackingUri = new URI(original);
+        }
       }
       }
 
 
       String runningUser = applicationReport.getUser();
       String runningUser = applicationReport.getUser();
@@ -311,8 +313,7 @@ public class WebAppProxyServlet extends HttpServlet {
             req.getQueryString(), true), runningUser, id);
             req.getQueryString(), true), runningUser, id);
         return;
         return;
       }
       }
-      
-      URI toFetch = new URI(req.getScheme(), 
+      URI toFetch = new URI(trackingUri.getScheme(), 
           trackingUri.getAuthority(),
           trackingUri.getAuthority(),
           StringHelper.ujoin(trackingUri.getPath(), rest), req.getQueryString(),
           StringHelper.ujoin(trackingUri.getPath(), rest), req.getQueryString(),
           null);
           null);