You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by as...@apache.org on 2015/12/13 08:27:20 UTC

[01/37] hadoop git commit: HADOOP-12321. Make JvmPauseMonitor an AbstractService. (Sunil G via Stevel) [includes HDFS-8947 MAPREDUCE-6462 and YARN-4072]

Repository: hadoop
Updated Branches:
  refs/heads/yarn-2877 742632e34 -> 7fb212e5e


HADOOP-12321. Make JvmPauseMonitor an AbstractService. (Sunil G via Stevel) [includes HDFS-8947 MAPREDUCE-6462 and YARN-4072]


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/65f39522
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/65f39522
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/65f39522

Branch: refs/heads/yarn-2877
Commit: 65f395226ba6cc3750a268a308e288b916f8df1e
Parents: 742632e
Author: Steve Loughran <st...@apache.org>
Authored: Sun Dec 6 17:42:56 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Sun Dec 6 17:43:35 2015 +0000

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../org/apache/hadoop/util/JvmPauseMonitor.java | 50 ++++++++------
 .../hadoop/metrics2/source/TestJvmMetrics.java  | 70 +++++++++++++++++++-
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java    |  3 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |  3 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  3 +-
 .../mapreduce/v2/hs/JobHistoryServer.java       |  7 +-
 .../mapreduce/v2/hs/TestJobHistoryServer.java   |  1 -
 hadoop-yarn-project/CHANGES.txt                 |  4 ++
 .../ApplicationHistoryServer.java               |  9 +--
 .../TestApplicationHistoryServer.java           |  1 -
 .../yarn/server/nodemanager/NodeManager.java    |  7 +-
 .../server/resourcemanager/ResourceManager.java |  9 +--
 .../yarn/server/webproxy/WebAppProxyServer.java |  9 +--
 15 files changed, 123 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index de4dad0..e7da77b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -635,6 +635,9 @@ Release 2.9.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-12321. Make JvmPauseMonitor an AbstractService.
+    (Sunil G via Stevel)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
index 73d7309..882e4a7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -43,21 +44,21 @@ import com.google.common.collect.Sets;
  * detected, the thread logs a message.
  */
 @InterfaceAudience.Private
-public class JvmPauseMonitor {
+public class JvmPauseMonitor extends AbstractService {
   private static final Log LOG = LogFactory.getLog(
       JvmPauseMonitor.class);
 
   /** The target sleep time */
   private static final long SLEEP_INTERVAL_MS = 500;
-  
+
   /** log WARN if we detect a pause longer than this threshold */
-  private final long warnThresholdMs;
+  private long warnThresholdMs;
   private static final String WARN_THRESHOLD_KEY =
       "jvm.pause.warn-threshold.ms";
   private static final long WARN_THRESHOLD_DEFAULT = 10000;
-  
+
   /** log INFO if we detect a pause longer than this threshold */
-  private final long infoThresholdMs;
+  private long infoThresholdMs;
   private static final String INFO_THRESHOLD_KEY =
       "jvm.pause.info-threshold.ms";
   private static final long INFO_THRESHOLD_DEFAULT = 1000;
@@ -65,29 +66,32 @@ public class JvmPauseMonitor {
   private long numGcWarnThresholdExceeded = 0;
   private long numGcInfoThresholdExceeded = 0;
   private long totalGcExtraSleepTime = 0;
-   
+
   private Thread monitorThread;
   private volatile boolean shouldRun = true;
 
-  public JvmPauseMonitor(Configuration conf) {
+  public JvmPauseMonitor() {
+    super(JvmPauseMonitor.class.getName());
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
     this.warnThresholdMs = conf.getLong(WARN_THRESHOLD_KEY, WARN_THRESHOLD_DEFAULT);
     this.infoThresholdMs = conf.getLong(INFO_THRESHOLD_KEY, INFO_THRESHOLD_DEFAULT);
+    super.serviceInit(conf);
   }
-  
-  public void start() {
-    Preconditions.checkState(monitorThread == null,
-        "Already started");
+
+  @Override
+  protected void serviceStart() throws Exception {
     monitorThread = new Daemon(new Monitor());
-    if (shouldRun) {
-      monitorThread.start();
-    } else {
-      LOG.warn("stop() was called before start() completed");
-    }
+    monitorThread.start();
+    super.serviceStart();
   }
-  
-  public void stop() {
+
+  @Override
+  protected void serviceStop() throws Exception {
     shouldRun = false;
-    if (isStarted()) {
+    if (monitorThread != null) {
       monitorThread.interrupt();
       try {
         monitorThread.join();
@@ -95,12 +99,13 @@ public class JvmPauseMonitor {
         Thread.currentThread().interrupt();
       }
     }
+    super.serviceStop();
   }
 
   public boolean isStarted() {
     return monitorThread != null;
   }
-  
+
   public long getNumGcWarnThreadholdExceeded() {
     return numGcWarnThresholdExceeded;
   }
@@ -212,8 +217,11 @@ public class JvmPauseMonitor {
    * with a 1GB heap will very quickly go into "GC hell" and result in
    * log messages about the GC pauses.
    */
+  @SuppressWarnings("resource")
   public static void main(String []args) throws Exception {
-    new JvmPauseMonitor(new Configuration()).start();
+    JvmPauseMonitor monitor = new JvmPauseMonitor();
+    monitor.init(new Configuration());
+    monitor.start();
     List<String> list = Lists.newArrayList();
     int i = 0;
     while (true) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java
index 3cb3384..69e7565 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java
@@ -18,7 +18,11 @@
 
 package org.apache.hadoop.metrics2.source;
 
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import static org.mockito.Mockito.*;
 import static org.apache.hadoop.test.MetricsAsserts.*;
@@ -26,6 +30,9 @@ import static org.apache.hadoop.test.MetricsAsserts.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.JvmPauseMonitor;
 
 import static org.apache.hadoop.metrics2.source.JvmMetricsInfo.*;
@@ -33,8 +40,23 @@ import static org.apache.hadoop.metrics2.impl.MsInfo.*;
 
 public class TestJvmMetrics {
 
-  @Test public void testPresence() {
-    JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(new Configuration());
+  @Rule
+  public Timeout timeout = new Timeout(30000);
+  private JvmPauseMonitor pauseMonitor;
+
+  /**
+   * Robust shutdown of the pause monitor if it hasn't been stopped already.
+   */
+  @After
+  public void teardown() {
+    ServiceOperations.stop(pauseMonitor);
+  }
+
+  @Test
+  public void testPresence() {
+    pauseMonitor = new JvmPauseMonitor();
+    pauseMonitor.init(new Configuration());
+    pauseMonitor.start();
     JvmMetrics jvmMetrics = new JvmMetrics("test", "test");
     jvmMetrics.setPauseMonitor(pauseMonitor);
     MetricsRecordBuilder rb = getMetrics(jvmMetrics);
@@ -54,4 +76,48 @@ public class TestJvmMetrics {
         verify(rb).addCounter(eq(info), anyLong());
     }
   }
+
+  @Test
+  public void testDoubleStop() throws Throwable {
+    pauseMonitor = new JvmPauseMonitor();
+    pauseMonitor.init(new Configuration());
+    pauseMonitor.start();
+    pauseMonitor.stop();
+    pauseMonitor.stop();
+  }
+
+  @Test
+  public void testDoubleStart() throws Throwable {
+    pauseMonitor = new JvmPauseMonitor();
+    pauseMonitor.init(new Configuration());
+    pauseMonitor.start();
+    pauseMonitor.start();
+    pauseMonitor.stop();
+  }
+
+  @Test
+  public void testStopBeforeStart() throws Throwable {
+    pauseMonitor = new JvmPauseMonitor();
+    try {
+      pauseMonitor.init(new Configuration());
+      pauseMonitor.stop();
+      pauseMonitor.start();
+      Assert.fail("Expected an exception, got " + pauseMonitor);
+    } catch (ServiceStateException e) {
+      GenericTestUtils.assertExceptionContains("cannot enter state", e);
+    }
+  }
+
+  @Test
+  public void testStopBeforeInit() throws Throwable {
+    pauseMonitor = new JvmPauseMonitor();
+    try {
+      pauseMonitor.stop();
+      pauseMonitor.init(new Configuration());
+      Assert.fail("Expected an exception, got " + pauseMonitor);
+    } catch (ServiceStateException e) {
+      GenericTestUtils.assertExceptionContains("cannot enter state", e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
index 05d806a..0282524 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
@@ -243,7 +243,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   @Override
   public void startDaemons() {
     if (pauseMonitor == null) {
-      pauseMonitor = new JvmPauseMonitor(config);
+      pauseMonitor = new JvmPauseMonitor();
+      pauseMonitor.init(config);
       pauseMonitor.start();
       metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 118df6e..642414f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -893,6 +893,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9414. Refactor reconfiguration of ClientDatanodeProtocol for
     reusability. (Xiaobing Zhou via Arpit Agarwal)
 
+    HDFS-8947. NameNode, DataNode and NFS gateway to support JvmPauseMonitor as
+    a service. (Sunil G via Stevel)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 15e556b..4b6e6e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1244,7 +1244,8 @@ public class DataNode extends ReconfigurableBase
     registerMXBean();
     initDataXceiver(conf);
     startInfoServer(conf);
-    pauseMonitor = new JvmPauseMonitor(conf);
+    pauseMonitor = new JvmPauseMonitor();
+    pauseMonitor.init(conf);
     pauseMonitor.start();
   
     // BlockPoolTokenSecretManager is required to create ipc server.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index c3f3017..acb4069 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -691,7 +691,8 @@ public class NameNode implements NameNodeStatusMXBean {
       httpServer.setFSImage(getFSImage());
     }
     
-    pauseMonitor = new JvmPauseMonitor(conf);
+    pauseMonitor = new JvmPauseMonitor();
+    pauseMonitor.init(conf);
     pauseMonitor.start();
     metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
index b5ac91a..c93c4cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
@@ -145,7 +145,8 @@ public class JobHistoryServer extends CompositeService {
 
     DefaultMetricsSystem.initialize("JobHistoryServer");
     JvmMetrics jm = JvmMetrics.initSingleton("JobHistoryServer", null);
-    pauseMonitor = new JvmPauseMonitor(getConfig());
+    pauseMonitor = new JvmPauseMonitor();
+    addService(pauseMonitor);
     jm.setPauseMonitor(pauseMonitor);
 
     super.serviceInit(config);
@@ -198,16 +199,12 @@ public class JobHistoryServer extends CompositeService {
 
   @Override
   protected void serviceStart() throws Exception {
-    pauseMonitor.start();
     super.serviceStart();
   }
   
   @Override
   protected void serviceStop() throws Exception {
     DefaultMetricsSystem.shutdown();
-    if (pauseMonitor != null) {
-      pauseMonitor.stop();
-    }
     super.serviceStop();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
index 32b2cff..c914474 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
@@ -77,7 +77,6 @@ public class TestJobHistoryServer {
     Configuration config = new Configuration();
     historyServer.init(config);
     assertEquals(STATE.INITED, historyServer.getServiceState());
-    assertEquals(6, historyServer.getServices().size());
     HistoryClientService historyService = historyServer.getClientService();
     assertNotNull(historyServer.getClientService());
     assertEquals(STATE.INITED, historyService.getServiceState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1fed6a6..632c888 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -63,6 +63,10 @@ Release 2.9.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    YARN-4072. ApplicationHistoryServer, WebAppProxyServer, NodeManager and
+    ResourceManager to support JvmPauseMonitor as a service.
+    (Sunil G via Stevel)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
index cda84a2..160ad8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
@@ -103,7 +103,8 @@ public class ApplicationHistoryServer extends CompositeService {
 
     DefaultMetricsSystem.initialize("ApplicationHistoryServer");
     JvmMetrics jm = JvmMetrics.initSingleton("ApplicationHistoryServer", null);
-    pauseMonitor = new JvmPauseMonitor(conf);
+    pauseMonitor = new JvmPauseMonitor();
+    addService(pauseMonitor);
     jm.setPauseMonitor(pauseMonitor);
     super.serviceInit(conf);
   }
@@ -116,9 +117,6 @@ public class ApplicationHistoryServer extends CompositeService {
       throw new YarnRuntimeException("Failed to login", ie);
     }
 
-    if (pauseMonitor != null) {
-      pauseMonitor.start();
-    }
     super.serviceStart();
     startWebApp();
   }
@@ -128,9 +126,6 @@ public class ApplicationHistoryServer extends CompositeService {
     if (webApp != null) {
       webApp.stop();
     }
-    if (pauseMonitor != null) {
-      pauseMonitor.stop();
-    }
     DefaultMetricsSystem.shutdown();
     super.serviceStop();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java
index 8dbccaa..b9d1c5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java
@@ -73,7 +73,6 @@ public class TestApplicationHistoryServer {
       historyServer = new ApplicationHistoryServer();
       historyServer.init(config);
       assertEquals(STATE.INITED, historyServer.getServiceState());
-      assertEquals(5, historyServer.getServices().size());
       ApplicationHistoryClientService historyService =
           historyServer.getClientService();
       assertNotNull(historyServer.getClientService());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 04e383f..a9a5411 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -344,7 +344,8 @@ public class NodeManager extends CompositeService
     dispatcher.register(NodeManagerEventType.class, this);
     addService(dispatcher);
 
-    pauseMonitor = new JvmPauseMonitor(conf);
+    pauseMonitor = new JvmPauseMonitor();
+    addService(pauseMonitor);
     metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
 
     DefaultMetricsSystem.initialize("NodeManager");
@@ -364,7 +365,6 @@ public class NodeManager extends CompositeService
     } catch (IOException e) {
       throw new YarnRuntimeException("Failed NodeManager login", e);
     }
-    pauseMonitor.start();
     super.serviceStart();
   }
 
@@ -376,9 +376,6 @@ public class NodeManager extends CompositeService
     try {
       super.serviceStop();
       DefaultMetricsSystem.shutdown();
-      if (pauseMonitor != null) {
-        pauseMonitor.stop();
-      }
     } finally {
       // YARN-3641: NM's services stop get failed shouldn't block the
       // release of NMLevelDBStore.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 01a1c8f..d392410 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -518,7 +518,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
       DefaultMetricsSystem.initialize("ResourceManager");
       JvmMetrics jm = JvmMetrics.initSingleton("ResourceManager", null);
-      pauseMonitor = new JvmPauseMonitor(conf);
+      pauseMonitor = new JvmPauseMonitor();
+      addService(pauseMonitor);
       jm.setPauseMonitor(pauseMonitor);
 
       // Initialize the Reservation system
@@ -574,8 +575,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
       // need events to move to further states.
       rmStore.start();
 
-      pauseMonitor.start();
-
       if(recoveryEnabled) {
         try {
           LOG.info("Recovery started");
@@ -602,10 +601,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
       super.serviceStop();
 
-      if (pauseMonitor != null) {
-        pauseMonitor.stop();
-      }
-
       DefaultMetricsSystem.shutdown();
       if (rmContext != null) {
         RMStateStore store = rmContext.getStateStore();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65f39522/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
index 4c1caa6..a5dd8f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
@@ -67,7 +67,8 @@ public class WebAppProxyServer extends CompositeService {
 
     DefaultMetricsSystem.initialize("WebAppProxyServer");
     JvmMetrics jm = JvmMetrics.initSingleton("WebAppProxyServer", null);
-    pauseMonitor = new JvmPauseMonitor(conf);
+    pauseMonitor = new JvmPauseMonitor();
+    addService(pauseMonitor);
     jm.setPauseMonitor(pauseMonitor);
 
     super.serviceInit(config);
@@ -75,9 +76,6 @@ public class WebAppProxyServer extends CompositeService {
 
   @Override
   protected void serviceStart() throws Exception {
-    if (pauseMonitor != null) {
-      pauseMonitor.start();
-    }
     super.serviceStart();
   }
 
@@ -85,9 +83,6 @@ public class WebAppProxyServer extends CompositeService {
   protected void serviceStop() throws Exception {
     super.serviceStop();
     DefaultMetricsSystem.shutdown();
-    if (pauseMonitor != null) {
-      pauseMonitor.stop();
-    }
   }
 
   /**


[19/37] hadoop git commit: YARN-4421. Remove dead code in RmAppImpl.RMAppRecoveredTransition. (Daniel Templeton via rohithsharmaks)

Posted by as...@apache.org.
YARN-4421. Remove dead code in RmAppImpl.RMAppRecoveredTransition. (Daniel Templeton via rohithsharmaks)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a5e2e1ec
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a5e2e1ec
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a5e2e1ec

Branch: refs/heads/yarn-2877
Commit: a5e2e1ecb06a3942903cb79f61f0f4bb02480f19
Parents: 15c3e7f
Author: rohithsharmaks <ro...@apache.org>
Authored: Wed Dec 9 11:31:51 2015 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Wed Dec 9 11:31:51 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../yarn/server/resourcemanager/rmapp/RMAppImpl.java      | 10 ----------
 2 files changed, 3 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5e2e1ec/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3935cde..2e0514b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1111,6 +1111,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4431. Not necessary to do unRegisterNM() if NM get stop due to failed to connect
     to RM. (Junping Du via rohithsharmaks)
 
+    YARN-4421. Remove dead code in RmAppImpl.RMAppRecoveredTransition.
+    (Daniel Templeton via rohithsharmaks)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5e2e1ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 3a9df67..7a2b717 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -977,16 +977,6 @@ public class RMAppImpl implements RMApp, Recoverable {
       // recover attempts
       app.recoverAppAttempts();
 
-      // Last attempt is in final state, return ACCEPTED waiting for last
-      // RMAppAttempt to send finished or failed event back.
-      if (app.currentAttempt != null
-          && (app.currentAttempt.getState() == RMAppAttemptState.KILLED
-              || app.currentAttempt.getState() == RMAppAttemptState.FINISHED
-              || (app.currentAttempt.getState() == RMAppAttemptState.FAILED
-                  && app.getNumFailedAppAttempts() == app.maxAppAttempts))) {
-        return RMAppState.ACCEPTED;
-      }
-
       // YARN-1507 is saving the application state after the application is
       // accepted. So after YARN-1507, an app is saved meaning it is accepted.
       // Thus we return ACCECPTED state on recovery.


[35/37] hadoop git commit: HDFS-9519. Some coding improvement in SecondaryNameNode#main. (Xiao Chen via Yongjun Zhang)

Posted by as...@apache.org.
HDFS-9519. Some coding improvement in SecondaryNameNode#main. (Xiao Chen via Yongjun Zhang)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2a4c7d4f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2a4c7d4f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2a4c7d4f

Branch: refs/heads/yarn-2877
Commit: 2a4c7d4facabb8b99d6dcbf4ccfe2afedf4fd445
Parents: f5a9114
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Dec 11 18:44:01 2015 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Dec 11 18:44:01 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt           |  3 +++
 .../hdfs/server/namenode/SecondaryNameNode.java       | 14 ++++++--------
 2 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a4c7d4f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1fda1d78..767bbcb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1742,6 +1742,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9532. Detailed exception info is lost in reportTo methods of
     ErrorReportAction and ReportBadBlockAction. (Yongjun Zhang)
 
+    HDFS-9519. Some coding improvement in SecondaryNameNode#main.
+    (Xiao Chen via Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a4c7d4f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index 9909d4a..acb2c8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -679,18 +679,16 @@ public class SecondaryNameNode implements Runnable,
       SecondaryNameNode secondary = null;
       secondary = new SecondaryNameNode(tconf, opts);
 
+      // SecondaryNameNode can be started in 2 modes:
+      // 1. run a command (i.e. checkpoint or geteditsize) then terminate
+      // 2. run as a daemon when {@link #parseArgs} yields no commands
       if (opts != null && opts.getCommand() != null) {
+        // mode 1
         int ret = secondary.processStartupCommand(opts);
         terminate(ret);
-      }
-
-      if (secondary != null) {
-        // The web server is only needed when starting SNN as a daemon,
-        // and not needed if called from shell command. Starting the web server
-        // from shell may fail when getting credentials, if the environment
-        // is not set up for it, which is most of the case.
+      } else {
+        // mode 2
         secondary.startInfoServer();
-
         secondary.startCheckpointThread();
         secondary.join();
       }


[06/37] hadoop git commit: Move HDFS-9129 from trunk to branch-2.9.0

Posted by as...@apache.org.
Move HDFS-9129 from trunk to branch-2.9.0


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fa9ea85
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fa9ea85
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fa9ea85

Branch: refs/heads/yarn-2877
Commit: 7fa9ea85d47dec1702f113151eb437d5e3155e75
Parents: c25a635
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Dec 7 13:47:40 2015 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Mon Dec 7 13:47:40 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fa9ea85/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 642414f..169968f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -206,9 +206,6 @@ Trunk (Unreleased)
     HDFS-8968. Erasure coding: a comprehensive I/O throughput benchmark tool.
     (Rui Li via zhz)
 
-    HDFS-9129. Move the safemode block count into BlockManager. (Mingliang Liu
-    via jing9)
-
     HDFS-8831. Trash Support for deletion in HDFS encryption zone. (xyao)
 
   OPTIMIZATIONS
@@ -896,6 +893,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-8947. NameNode, DataNode and NFS gateway to support JvmPauseMonitor as
     a service. (Sunil G via Stevel)
 
+    HDFS-9129. Move the safemode block count into BlockManager. (Mingliang Liu
+    via jing9)
+
   OPTIMIZATIONS
 
   BUG FIXES


[18/37] hadoop git commit: YARN-4431. Not necessary to do unRegisterNM() if NM get stop due to failed to connect to RM. (Junpin Du via rohithsharmaks)

Posted by as...@apache.org.
YARN-4431. Not necessary to do unRegisterNM() if NM get stop due to failed to connect to RM. (Junpin Du via rohithsharmaks)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/15c3e7ff
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/15c3e7ff
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/15c3e7ff

Branch: refs/heads/yarn-2877
Commit: 15c3e7ffe3d1c57ad36afd993f09fc47889c93bd
Parents: ada9c2c
Author: rohithsharmaks <ro...@apache.org>
Authored: Wed Dec 9 10:50:43 2015 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Wed Dec 9 10:50:43 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 3 +++
 .../hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java   | 5 ++++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/15c3e7ff/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c7eae9f..3935cde 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1108,6 +1108,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4392. ApplicationCreatedEvent event time resets after RM restart/failover.
     (Naganarasimha G R and Xuan Gong via xgong)
 
+    YARN-4431. Not necessary to do unRegisterNM() if NM get stop due to failed to connect
+    to RM. (Junping Du via rohithsharmaks)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15c3e7ff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 34267b3..ba915c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -134,6 +134,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   private Runnable statusUpdaterRunnable;
   private Thread  statusUpdater;
+  private boolean failedToConnect = false;
   private long rmIdentifier = ResourceManagerConstants.RM_INVALID_IDENTIFIER;
   private boolean registeredWithRM = false;
   Set<ContainerId> pendingContainersToRemove = new HashSet<ContainerId>();
@@ -241,7 +242,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     // the isStopped check is for avoiding multiple unregistrations.
     if (this.registeredWithRM && !this.isStopped
         && !isNMUnderSupervisionWithRecoveryEnabled()
-        && !context.getDecommissioned()) {
+        && !context.getDecommissioned() && !failedToConnect) {
       unRegisterNM();
     }
     // Interrupt the updater.
@@ -823,6 +824,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             //catch and throw the exception if tried MAX wait time to connect RM
             dispatcher.getEventHandler().handle(
                 new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
+            // failed to connect to RM.
+            failedToConnect = true;
             throw new YarnRuntimeException(e);
           } catch (Throwable e) {
 


[20/37] hadoop git commit: HADOOP-12618. Fix NPE in TestSequenceFile. (Brahma Reddy Battula via umamahesh)

Posted by as...@apache.org.
HADOOP-12618. Fix NPE in TestSequenceFile. (Brahma Reddy Battula via umamahesh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ad53c520
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ad53c520
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ad53c520

Branch: refs/heads/yarn-2877
Commit: ad53c520630847834b3e68b7f41aa88ee10b3300
Parents: a5e2e1e
Author: Uma Mahesh <um...@apache.org>
Authored: Tue Dec 8 22:39:17 2015 -0800
Committer: Uma Mahesh <um...@apache.org>
Committed: Tue Dec 8 22:39:17 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  2 ++
 .../org/apache/hadoop/io/TestSequenceFile.java     | 17 +++++++++++++++--
 2 files changed, 17 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad53c520/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 0f505dd..6bd6267 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1000,6 +1000,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12600. FileContext and AbstractFileSystem should be annotated as a
     Stable interface. (cnauroth)
 
+    HADOOP-12618. Fix NPE in TestSequenceFile. (Brahma Reddy Battula via umamahesh)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad53c520/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java
index e127928..4cb4e13 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFile.java
@@ -130,6 +130,7 @@ public class TestSequenceFile {
     }
   }
 
+  @SuppressWarnings("deprecation")
   private void writeTest(FileSystem fs, int count, int seed, Path file, 
                                 CompressionType compressionType, CompressionCodec codec)
     throws IOException {
@@ -150,6 +151,7 @@ public class TestSequenceFile {
     writer.close();
   }
 
+  @SuppressWarnings("deprecation")
   private void readTest(FileSystem fs, int count, int seed, Path file)
     throws IOException {
     LOG.debug("reading " + count + " records");
@@ -216,6 +218,7 @@ public class TestSequenceFile {
     LOG.info("done sorting " + count + " debug");
   }
 
+  @SuppressWarnings("deprecation")
   private void checkSort(FileSystem fs, int count, int seed, Path file)
     throws IOException {
     LOG.info("sorting " + count + " records in memory for debug");
@@ -253,6 +256,7 @@ public class TestSequenceFile {
     LOG.debug("sucessfully checked " + count + " records");
   }
 
+  @SuppressWarnings("deprecation")
   private void mergeTest(FileSystem fs, int count, int seed, Path file, 
                                 CompressionType compressionType,
                                 boolean fast, int factor, int megabytes)
@@ -375,6 +379,7 @@ public class TestSequenceFile {
   }
   
   
+  @SuppressWarnings("deprecation")
   private SequenceFile.Metadata readMetadata(FileSystem fs, Path file)
     throws IOException {
     LOG.info("reading file: " + file.toString());
@@ -384,6 +389,7 @@ public class TestSequenceFile {
     return meta;
   }
 
+  @SuppressWarnings("deprecation")
   private void writeMetadataTest(FileSystem fs, int count, int seed, Path file, 
                                         CompressionType compressionType, CompressionCodec codec, SequenceFile.Metadata metadata)
     throws IOException {
@@ -413,6 +419,7 @@ public class TestSequenceFile {
     sorter.sort(new Path[] { unsortedFile }, sortedFile, false);
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testClose() throws IOException {
     Configuration conf = new Configuration();
@@ -470,6 +477,7 @@ public class TestSequenceFile {
    * Test that makes sure the FileSystem passed to createWriter
    * @throws Exception
    */
+  @SuppressWarnings("deprecation")
   @Test
   public void testCreateUsesFsArg() throws Exception {
     FileSystem fs = FileSystem.getLocal(conf);
@@ -499,6 +507,7 @@ public class TestSequenceFile {
     }
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testCloseForErroneousSequenceFile()
     throws IOException {
@@ -555,6 +564,7 @@ public class TestSequenceFile {
    * already created
    * @throws IOException
    */
+  @SuppressWarnings("deprecation")
   @Test
   public void testCreateWriterOnExistingFile() throws IOException {
     Configuration conf = new Configuration();
@@ -568,6 +578,7 @@ public class TestSequenceFile {
         CompressionType.NONE, null, new Metadata());
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testRecursiveSeqFileCreate() throws IOException {
     FileSystem fs = FileSystem.getLocal(conf);
@@ -661,7 +672,7 @@ public class TestSequenceFile {
     Path file = null;
     int seed = new Random().nextInt();
 
-    String usage = "Usage: SequenceFile " +
+    String usage = "Usage: testsequencefile " +
       "[-count N] " + 
       "[-seed #] [-check] [-compressType <NONE|RECORD|BLOCK>] " + 
       "-codec <compressionCodec> " + 
@@ -751,7 +762,9 @@ public class TestSequenceFile {
         test.checkSort(fs, count, seed, file);
       }
     } finally {
-      fs.close();
+      if (fs != null) {
+        fs.close();
+      }
     }
   }
 }


[32/37] hadoop git commit: Revert "HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)"

Posted by as...@apache.org.
Revert "HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)"

This reverts commit 832b3cbde1c2f77b04c93188e3a94420974090cf.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/576b569b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/576b569b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/576b569b

Branch: refs/heads/yarn-2877
Commit: 576b569b6c97bd5f57e52efdabdf8c2fa996a524
Parents: b7b2923
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Dec 11 10:59:42 2015 -0800
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Dec 11 10:59:42 2015 -0800

----------------------------------------------------------------------
 .../fs/CommonConfigurationKeysPublic.java       |  3 -
 .../hadoop/security/UserGroupInformation.java   | 27 +-------
 .../src/main/resources/core-default.xml         |  6 --
 .../security/TestUserGroupInformation.java      | 46 +------------
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 70 ++++++--------------
 .../hdfs/web/resources/DelegationParam.java     |  6 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 -
 .../web/resources/NamenodeWebHdfsMethods.java   |  4 ++
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      | 69 ++++---------------
 .../apache/hadoop/hdfs/web/TestWebHdfsUrl.java  | 33 ++-------
 10 files changed, 49 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 648ad59..c9f758b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -311,9 +311,6 @@ public class CommonConfigurationKeysPublic {
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_SECURITY_DNS_NAMESERVER_KEY =
     "hadoop.security.dns.nameserver";
-  /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
-  public static final String HADOOP_TOKEN_FILES =
-      "hadoop.token.files";
 
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index a9871a5..483420c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.security;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 import java.io.File;
@@ -249,9 +248,9 @@ public class UserGroupInformation {
   
   /**Environment variable pointing to the token cache file*/
   public static final String HADOOP_TOKEN_FILE_LOCATION = 
-      "HADOOP_TOKEN_FILE_LOCATION";
-
-  /**
+    "HADOOP_TOKEN_FILE_LOCATION";
+  
+  /** 
    * A method to initialize the fields that depend on a configuration.
    * Must be called before useKerberos or groups is used.
    */
@@ -822,26 +821,6 @@ public class UserGroupInformation {
       }
       loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
 
-      String tokenFileLocation = System.getProperty(HADOOP_TOKEN_FILES);
-      if (tokenFileLocation == null) {
-        tokenFileLocation = conf.get(HADOOP_TOKEN_FILES);
-      }
-      if (tokenFileLocation != null) {
-        String[] tokenFileNames = tokenFileLocation.split("\\s*,\\s*+");
-        for (String tokenFileName: tokenFileNames) {
-          if (tokenFileName.length() > 0) {
-            File tokenFile = new File(tokenFileName);
-            if (tokenFile.exists() && tokenFile.isFile()) {
-              Credentials cred = Credentials.readTokenStorageFile(
-                  tokenFile, conf);
-              loginUser.addCredentials(cred);
-            } else {
-              LOG.info("tokenFile("+tokenFileName+") does not exist");
-            }
-          }
-        }
-      }
-
       String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
       if (fileLocation != null) {
         // Load the token storage file and put all of the tokens into the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 4b69a65..c8f7f71 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -466,12 +466,6 @@ for ldap providers in the same way as above does.
   <description>Maps kerberos principals to local user names</description>
 </property>
 
-<property>
-  <name>hadoop.token.files</name>
-  <value></value>
-  <description>List of token cache files that have delegation tokens for hadoop service</description>
-</property>
-
 <!-- i/o properties -->
 <property>
   <name>io.file.buffer.size</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index 0539a03..54cfc2d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -18,7 +18,6 @@ package org.apache.hadoop.security;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
@@ -36,7 +35,6 @@ import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.lang.reflect.Method;
@@ -863,7 +861,7 @@ public class TestUserGroupInformation {
 
     // Ensure only non-private tokens are returned
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();
-    assertEquals(3, tokens.size());
+    assertEquals(1, tokens.size());
   }
 
   /**
@@ -930,46 +928,4 @@ public class TestUserGroupInformation {
       }
     }
   }
-
-  @Test (timeout = 30000)
-  public void testExternalTokenFiles() throws Exception {
-    StringBuilder tokenFullPathnames = new StringBuilder();
-    String tokenFilenames = "token1,token2";
-    String tokenFiles[] = tokenFilenames.split("\\s*,\\s*+");
-    final File testDir = new File("target",
-        TestUserGroupInformation.class.getName() + "-tmpDir").getAbsoluteFile();
-    String testDirPath = testDir.getAbsolutePath();
-
-    // create path for token files
-    for (String tokenFile: tokenFiles) {
-      if (tokenFullPathnames.length() > 0) {
-        tokenFullPathnames.append(",");
-      }
-      tokenFullPathnames.append(testDirPath).append("/").append(tokenFile);
-    }
-
-    // create new token and store it
-    TestTokenIdentifier tokenId = new TestTokenIdentifier();
-    Credentials cred1 = new Credentials();
-    Token<TestTokenIdentifier> token1 = new Token<TestTokenIdentifier>(
-            tokenId.getBytes(), "password".getBytes(),
-            tokenId.getKind(), new Text("token-service1"));
-    cred1.addToken(token1.getService(), token1);
-    cred1.writeTokenStorageFile(new Path(testDirPath, tokenFiles[0]), conf);
-
-    Credentials cred2 = new Credentials();
-    Token<TestTokenIdentifier> token2 = new Token<TestTokenIdentifier>(
-            tokenId.getBytes(), "password".getBytes(),
-            tokenId.getKind(), new Text("token-service2"));
-    cred2.addToken(token2.getService(), token2);
-    cred2.writeTokenStorageFile(new Path(testDirPath, tokenFiles[1]), conf);
-
-    // set property for token external token files
-    System.setProperty("hadoop.token.files", tokenFullPathnames.toString());
-    UserGroupInformation.setLoginUser(null);
-    UserGroupInformation tokenUgi = UserGroupInformation.getLoginUser();
-    Collection<Token<?>> credsugiTokens = tokenUgi.getTokens();
-    assertTrue(credsugiTokens.contains(token1));
-    assertTrue(credsugiTokens.contains(token2));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index c2a7ef8..4049b80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -231,7 +231,7 @@ public class WebHdfsFileSystem extends FileSystem
   // the first getAuthParams() for a non-token op will either get the
   // internal token from the ugi or lazy fetch one
   protected synchronized Token<?> getDelegationToken() throws IOException {
-    if (delegationToken == null) {
+    if (canRefreshDelegationToken && delegationToken == null) {
       Token<?> token = tokenSelector.selectToken(
           new Text(getCanonicalServiceName()), ugi.getTokens());
       // ugi tokens are usually indicative of a task which can't
@@ -241,13 +241,11 @@ public class WebHdfsFileSystem extends FileSystem
         LOG.debug("Using UGI token: {}", token);
         canRefreshDelegationToken = false;
       } else {
-        if (canRefreshDelegationToken) {
-          token = getDelegationToken(null);
-          if (token != null) {
-            LOG.debug("Fetched new token: {}", token);
-          } else { // security is disabled
-            canRefreshDelegationToken = false;
-          }
+        token = getDelegationToken(null);
+        if (token != null) {
+          LOG.debug("Fetched new token: {}", token);
+        } else { // security is disabled
+          canRefreshDelegationToken = false;
         }
       }
       setDelegationToken(token);
@@ -259,7 +257,6 @@ public class WebHdfsFileSystem extends FileSystem
   synchronized boolean replaceExpiredDelegationToken() throws IOException {
     boolean replaced = false;
     if (canRefreshDelegationToken) {
-      this.delegationToken = null;
       Token<?> token = getDelegationToken(null);
       LOG.debug("Replaced expired token: {}", token);
       setDelegationToken(token);
@@ -1349,7 +1346,7 @@ public class WebHdfsFileSystem extends FileSystem
     final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS;
     return new FsPathResponseRunner<FileStatus[]>(op, f) {
       @Override
-      FileStatus[] decodeResponse(Map<?, ?> json) {
+      FileStatus[] decodeResponse(Map<?,?> json) {
         final Map<?, ?> rootmap =
             (Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
         final List<?> array = JsonUtilClient.getList(rootmap,
@@ -1370,34 +1367,18 @@ public class WebHdfsFileSystem extends FileSystem
   }
 
   @Override
-  public synchronized Token<DelegationTokenIdentifier> getDelegationToken(
+  public Token<DelegationTokenIdentifier> getDelegationToken(
       final String renewer) throws IOException {
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
-    Token<DelegationTokenIdentifier> token = null;
-
-    if (delegationToken == null) {
-      token =
-          new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
-              op, null, new RenewerParam(renewer)) {
+    Token<DelegationTokenIdentifier> token =
+        new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
+            op, null, new RenewerParam(renewer)) {
           @Override
-          Token<DelegationTokenIdentifier> decodeResponse(Map<?, ?> json)
+          Token<DelegationTokenIdentifier> decodeResponse(Map<?,?> json)
               throws IOException {
             return JsonUtilClient.toDelegationToken(json);
           }
         }.run();
-    } else {
-      token =
-          new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
-              op, null, new RenewerParam(renewer),
-              new DelegationParam(delegationToken.encodeToUrlString())) {
-          @Override
-          Token<DelegationTokenIdentifier> decodeResponse(Map<?, ?> json)
-              throws IOException {
-            return JsonUtilClient.toDelegationToken(json);
-          }
-        }.run();
-    }
-
     if (token != null) {
       token.setService(tokenServiceName);
     } else {
@@ -1425,26 +1406,13 @@ public class WebHdfsFileSystem extends FileSystem
   public synchronized long renewDelegationToken(final Token<?> token
   ) throws IOException {
     final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
-
-    if (delegationToken == null) {
-      return new FsPathResponseRunner<Long>(op, null,
-          new TokenArgumentParam(token.encodeToUrlString())) {
-        @Override
-        Long decodeResponse(Map<?, ?> json) throws IOException {
-          return ((Number) json.get("long")).longValue();
-        }
-      }.run();
-    } else {
-      return new FsPathResponseRunner<Long>(op, null,
-          new TokenArgumentParam(token.encodeToUrlString()),
-          new DelegationParam(delegationToken.encodeToUrlString())) {
-        @Override
-        Long decodeResponse(Map<?, ?> json) throws IOException {
-          return ((Number) json.get("long")).longValue();
-        }
-      }.run();
-    }
-
+    return new FsPathResponseRunner<Long>(op, null,
+        new TokenArgumentParam(token.encodeToUrlString())) {
+      @Override
+      Long decodeResponse(Map<?,?> json) throws IOException {
+        return ((Number) json.get("long")).longValue();
+      }
+    }.run();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
index fb129d8..5329580 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.web.resources;
 
+import org.apache.hadoop.security.UserGroupInformation;
+
 /** Represents delegation token used for authentication. */
 public class DelegationParam extends StringParam {
   /** Parameter name. */
@@ -31,8 +33,8 @@ public class DelegationParam extends StringParam {
    * @param str a string representation of the parameter value.
    */
   public DelegationParam(final String str) {
-    super(DOMAIN, str != null && !str.equals(DEFAULT)? str: null);
-
+    super(DOMAIN, UserGroupInformation.isSecurityEnabled()
+        && str != null && !str.equals(DEFAULT)? str: null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c1a323b..f84e1e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -55,9 +55,6 @@ Trunk (Unreleased)
     HDFS-9057. allow/disallow snapshots via webhdfs
     (Bramma Reddy Battula via vinayakumarb)
 
-    HDFS-9525. hadoop utilities need to support provided delegation tokens
-    (HeeSoo Kim via aw)
-
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 5e602b5..4626507 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -895,6 +895,10 @@ public class NamenodeWebHdfsMethods {
     }
     case GETDELEGATIONTOKEN:
     {
+      if (delegation.getValue() != null) {
+        throw new IllegalArgumentException(delegation.getName()
+            + " parameter is not null.");
+      }
       final Token<? extends TokenIdentifier> token = generateDelegationToken(
           namenode, ugi, renewer.getValue());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
index b17cb4d..5e8568c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
@@ -296,59 +296,7 @@ public class TestWebHdfsTokens {
       }
     }
   }
-
-  @Test
-  public void testReuseToken() throws Exception {
-    MiniDFSCluster cluster = null;
-
-    UserGroupInformation loginUgi = UserGroupInformation.createUserForTesting(
-        "LoginUser", new String[]{"supergroup"});
-
-    try {
-      final Configuration clusterConf = new HdfsConfiguration(conf);
-      SecurityUtil.setAuthenticationMethod(SIMPLE, clusterConf);
-      clusterConf.setBoolean(DFSConfigKeys
-          .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
-      UserGroupInformation.setConfiguration(clusterConf);
-      UserGroupInformation.setLoginUser(loginUgi);
-
-      cluster = new MiniDFSCluster.Builder(clusterConf).numDataNodes(0).build();
-      cluster.waitActive();
-
-      /* create SIMPLE client connection */
-      SecurityUtil.setAuthenticationMethod(SIMPLE, clusterConf);
-      UserGroupInformation.setConfiguration(clusterConf);
-      UserGroupInformation simpleUgi = UserGroupInformation.createUserForTesting(
-          "testUser", new String[]{"supergroup"});
-      final WebHdfsFileSystem simpleFs = WebHdfsTestUtil.getWebHdfsFileSystemAs
-          (simpleUgi, clusterConf, "webhdfs");
-
-      /* create KERBEROS client connection */
-      SecurityUtil.setAuthenticationMethod(KERBEROS, clusterConf);
-      UserGroupInformation.setConfiguration(clusterConf);
-      UserGroupInformation krbUgi = UserGroupInformation.createUserForTesting(
-          "testUser", new String[]{"supergroup"});
-      final WebHdfsFileSystem krbFs = WebHdfsTestUtil.getWebHdfsFileSystemAs
-              (krbUgi, clusterConf, "webhdfs");
-
-      // 1. Get initial token through kerberos client connection
-      Token<DelegationTokenIdentifier> krbToken
-        = krbFs.getDelegationToken(null);
-      Assert.assertNotNull(krbToken);
-
-      // 2. Get token with previous token which gets from kerberos connection
-      //    through SIMPLE client connection.
-      simpleFs.setDelegationToken(krbToken);
-      Token<?> simpleToken =  simpleFs.getDelegationToken();
-      Assert.assertNotNull(simpleToken);
-      Assert.assertEquals(krbToken.getService(), simpleToken.getService());
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
+  
   @SuppressWarnings("unchecked")
   private void validateLazyTokenFetch(final Configuration clusterConf) throws Exception{
     final String testUser = "DummyUser";
@@ -360,6 +308,16 @@ public class TestWebHdfsTokens {
         return spy((WebHdfsFileSystem) FileSystem.newInstance(uri, clusterConf));
 	  }
     });
+    // verify token ops don't get a token
+    Assert.assertNull(fs.getRenewToken());
+    Token<?> token = fs.getDelegationToken(null);
+    fs.renewDelegationToken(token);
+    fs.cancelDelegationToken(token);
+    verify(fs, never()).getDelegationToken();
+    verify(fs, never()).replaceExpiredDelegationToken();
+    verify(fs, never()).setDelegationToken(any(Token.class));
+    Assert.assertNull(fs.getRenewToken());
+    reset(fs);
 
     // verify first non-token op gets a token
     final Path p = new Path("/f");
@@ -368,8 +326,8 @@ public class TestWebHdfsTokens {
     verify(fs, never()).replaceExpiredDelegationToken();
     verify(fs, times(1)).getDelegationToken(anyString());
     verify(fs, times(1)).setDelegationToken(any(Token.class));
-    Token<?> token = fs.getRenewToken();
-    Assert.assertNotNull(token);
+    token = fs.getRenewToken();
+    Assert.assertNotNull(token);      
     Assert.assertEquals(testUser, getTokenOwner(token));
     Assert.assertEquals(fs.getTokenKind(), token.getKind());
     reset(fs);
@@ -463,7 +421,6 @@ public class TestWebHdfsTokens {
     verify(fs, times(1)).cancelDelegationToken(eq(token2));
 
     // add a token to ugi for a new fs, verify it uses that token
-    fs.setDelegationToken(null);
     token = fs.getDelegationToken(null);
     ugi.addToken(token);
     fs = ugi.doAs(new PrivilegedExceptionAction<WebHdfsFileSystem>() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/576b569b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
index ba1c8a2..2913a97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
@@ -195,33 +195,9 @@ public class TestWebHdfsUrl {
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new DelegationParam(tokenString).toString()
-        },
-        fileStatusUrl);
-
-    // send user with delegationToken
-    getTokenUrl = webhdfs.toUrl(GetOpParam.Op.GETDELEGATIONTOKEN,
-        fsPath, new DelegationParam(tokenString));
-    checkQueryParams(
-        new String[]{
-            GetOpParam.Op.GETDELEGATIONTOKEN.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString(),
-            new DelegationParam(tokenString).toString()
-        },
-        getTokenUrl);
-
-    // send user with delegationToken
-    renewTokenUrl = webhdfs.toUrl(PutOpParam.Op.RENEWDELEGATIONTOKEN,
-        fsPath, new TokenArgumentParam(tokenString),
-        new DelegationParam(tokenString));
-    checkQueryParams(
-        new String[]{
-            PutOpParam.Op.RENEWDELEGATIONTOKEN.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString(),
-            new TokenArgumentParam(tokenString).toString(),
-            new DelegationParam(tokenString).toString()
+            new UserParam(ugi.getShortUserName()).toString()
         },
-        renewTokenUrl);
+        fileStatusUrl);    
   }
 
   @Test(timeout=60000)
@@ -298,13 +274,14 @@ public class TestWebHdfsUrl {
             new TokenArgumentParam(tokenString).toString()
         },
         cancelTokenUrl);
-
+    
     // send real+effective
     fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new DelegationParam(tokenString).toString()
+            new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+            new DoAsParam(ugi.getShortUserName()).toString()
         },
         fileStatusUrl);    
   }


[22/37] hadoop git commit: HADOOP-12625. Add a config to disable the /logs endpoints (rkanter)

Posted by as...@apache.org.
HADOOP-12625. Add a config to disable the /logs endpoints (rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e27fffdb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e27fffdb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e27fffdb

Branch: refs/heads/yarn-2877
Commit: e27fffdb9036d9ed9759d59b964020423b982e22
Parents: 50edcb9
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Dec 9 14:32:20 2015 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Dec 9 14:32:20 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  2 +
 .../fs/CommonConfigurationKeysPublic.java       |  6 ++
 .../org/apache/hadoop/http/HttpServer2.java     |  8 +-
 .../src/main/resources/core-default.xml         | 10 +++
 .../apache/hadoop/http/TestHttpServerLogs.java  | 80 ++++++++++++++++++++
 5 files changed, 104 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27fffdb/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 6bd6267..61ef7ba 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -641,6 +641,8 @@ Release 2.9.0 - UNRELEASED
     HADOOP-12321. Make JvmPauseMonitor an AbstractService.
     (Sunil G via Stevel)
 
+    HADOOP-12625. Add a config to disable the /logs endpoints (rkanter)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27fffdb/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 32dfc7e..648ad59 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -407,5 +407,11 @@ public class CommonConfigurationKeysPublic {
       "hadoop.shell.safely.delete.limit.num.files";
   public static final long HADOOP_SHELL_SAFELY_DELETE_LIMIT_NUM_FILES_DEFAULT =
       100;
+
+  /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
+  public static final String HADOOP_HTTP_LOGS_ENABLED =
+      "hadoop.http.logs.enabled";
+  /** Defalt value for HADOOP_HTTP_LOGS_ENABLED */
+  public static final boolean HADOOP_HTTP_LOGS_ENABLED_DEFAULT = true;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27fffdb/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index e347fd8..b134ba6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -507,9 +507,13 @@ public final class HttpServer2 implements FilterContainer {
    */
   protected void addDefaultApps(ContextHandlerCollection parent,
       final String appDir, Configuration conf) throws IOException {
-    // set up the context for "/logs/" if "hadoop.log.dir" property is defined.
+    // set up the context for "/logs/" if "hadoop.log.dir" property is defined
+    // and it's enabled.
     String logDir = System.getProperty("hadoop.log.dir");
-    if (logDir != null) {
+    boolean logsEnabled = conf.getBoolean(
+        CommonConfigurationKeys.HADOOP_HTTP_LOGS_ENABLED,
+        CommonConfigurationKeys.HADOOP_HTTP_LOGS_ENABLED_DEFAULT);
+    if (logDir != null && logsEnabled) {
       Context logContext = new Context(parent, "/logs");
       logContext.setResourceBase(logDir);
       logContext.addServlet(AdminAuthorizedServlet.class, "/*");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27fffdb/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 318ab6f..4b69a65 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2056,4 +2056,14 @@ for ldap providers in the same way as above does.
     <description>The class names of the Span Receivers to use for Hadoop.
     </description>
   </property>
+
+  <property>
+    <description>
+      Enable the "/logs" endpoint on all Hadoop daemons, which serves local
+      logs, but may be considered a security risk due to it listing the contents
+      of a directory.
+    </description>
+    <name>hadoop.http.logs.enabled</name>
+    <value>true</value>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e27fffdb/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLogs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLogs.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLogs.java
new file mode 100644
index 0000000..46ea095
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLogs.java
@@ -0,0 +1,80 @@
+/**
+ * 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.http;
+
+import org.apache.commons.httpclient.HttpStatus;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.http.resource.JerseyResource;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+public class TestHttpServerLogs extends HttpServerFunctionalTest {
+  static final Log LOG = LogFactory.getLog(TestHttpServerLogs.class);
+  private static HttpServer2 server;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+  }
+
+  private void startServer(Configuration conf) throws Exception {
+    server = createTestServer(conf);
+    server.addJerseyResourcePackage(
+        JerseyResource.class.getPackage().getName(), "/jersey/*");
+    server.start();
+    baseUrl = getServerURL(server);
+    LOG.info("HTTP server started: "+ baseUrl);
+  }
+
+  @AfterClass
+  public static void cleanup() throws Exception {
+    if (server != null && server.isAlive()) {
+      server.stop();
+    }
+  }
+
+  @Test
+  public void testLogsEnabled() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeysPublic.HADOOP_HTTP_LOGS_ENABLED, true);
+    startServer(conf);
+    URL url = new URL("http://"
+        + NetUtils.getHostPortString(server.getConnectorAddress(0)) + "/logs");
+    HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+    assertEquals(HttpStatus.SC_OK, conn.getResponseCode());
+  }
+
+  @Test
+  public void testLogsDisabled() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeysPublic.HADOOP_HTTP_LOGS_ENABLED, false);
+    startServer(conf);
+    URL url = new URL(baseUrl + "/logs");
+    HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+    assertEquals(HttpStatus.SC_NOT_FOUND, conn.getResponseCode());
+  }
+}


[17/37] hadoop git commit: HADOOP-12617. SPNEGO authentication request to non-default realm gets default realm name inserted in target server principal. (mattf)

Posted by as...@apache.org.
HADOOP-12617. SPNEGO authentication request to non-default realm gets default realm name inserted in target server principal. (mattf)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ada9c2c4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ada9c2c4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ada9c2c4

Branch: refs/heads/yarn-2877
Commit: ada9c2c410c15e95d0a21ea2941986195606aad8
Parents: 7e47151
Author: mattf <ma...@apache.org>
Authored: Tue Dec 8 17:27:50 2015 -0800
Committer: mattf <ma...@apache.org>
Committed: Tue Dec 8 17:27:50 2015 -0800

----------------------------------------------------------------------
 .../authentication/util/KerberosUtil.java       | 87 ++++++++++++++++++--
 .../authentication/util/TestKerberosUtil.java   | 31 +++++--
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 3 files changed, 107 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ada9c2c4/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
index 0e8d8db..3d7b00d 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java
@@ -42,7 +42,7 @@ public class KerberosUtil {
 
   /* Return the Kerberos login module name */
   public static String getKrb5LoginModuleName() {
-    return System.getProperty("java.vendor").contains("IBM")
+    return (IBM_JAVA)
       ? "com.ibm.security.auth.module.Krb5LoginModule"
       : "com.sun.security.auth.module.Krb5LoginModule";
   }
@@ -72,7 +72,7 @@ public class KerberosUtil {
     Class<?> classRef;
     Method getInstanceMethod;
     Method getDefaultRealmMethod;
-    if (System.getProperty("java.vendor").contains("IBM")) {
+    if (IBM_JAVA) {
       classRef = Class.forName("com.ibm.security.krb5.internal.Config");
     } else {
       classRef = Class.forName("sun.security.krb5.Config");
@@ -83,17 +83,79 @@ public class KerberosUtil {
          new Class[0]);
     return (String)getDefaultRealmMethod.invoke(kerbConf, new Object[0]);
   }
-  
+
+  public static String getDefaultRealmProtected() {
+    String realmString = null;
+    try {
+      realmString = getDefaultRealm();
+    } catch (RuntimeException rte) {
+      //silently catch everything
+    } catch (Exception e) {
+      //silently return null
+    }
+    return realmString;
+  }
+
+  /*
+   * For a Service Host Principal specification, map the host's domain
+   * to kerberos realm, as specified by krb5.conf [domain_realm] mappings.
+   * Unfortunately the mapping routines are private to the security.krb5
+   * package, so have to construct a PrincipalName instance to derive the realm.
+   *
+   * Many things can go wrong with Kerberos configuration, and this is not
+   * the place to be throwing exceptions to help debug them.  Nor do we choose
+   * to make potentially voluminous logs on every call to a communications API.
+   * So we simply swallow all exceptions from the underlying libraries and
+   * return null if we can't get a good value for the realmString.
+   *
+   * @param shortprinc A service principal name with host fqdn as instance, e.g.
+   *     "HTTP/myhost.mydomain"
+   * @return String value of Kerberos realm, mapped from host fqdn
+   *     May be default realm, or may be null.
+   */
+  public static String getDomainRealm(String shortprinc) {
+    Class<?> classRef;
+    Object principalName; //of type sun.security.krb5.PrincipalName or IBM equiv
+    String realmString = null;
+    try {
+      if (IBM_JAVA) {
+        classRef = Class.forName("com.ibm.security.krb5.PrincipalName");
+      } else {
+        classRef = Class.forName("sun.security.krb5.PrincipalName");
+      }
+      int tKrbNtSrvHst = classRef.getField("KRB_NT_SRV_HST").getInt(null);
+      principalName = classRef.getConstructor(String.class, int.class).
+          newInstance(shortprinc, tKrbNtSrvHst);
+      realmString = (String)classRef.getMethod("getRealmString", new Class[0]).
+          invoke(principalName, new Object[0]);
+    } catch (RuntimeException rte) {
+      //silently catch everything
+    } catch (Exception e) {
+      //silently return default realm (which may itself be null)
+    }
+    if (null == realmString || realmString.equals("")) {
+      return getDefaultRealmProtected();
+    } else {
+      return realmString;
+    }
+  }
+
   /* Return fqdn of the current host */
   static String getLocalHostName() throws UnknownHostException {
     return InetAddress.getLocalHost().getCanonicalHostName();
   }
   
   /**
-   * Create Kerberos principal for a given service and hostname. It converts
+   * Create Kerberos principal for a given service and hostname,
+   * inferring realm from the fqdn of the hostname. It converts
    * hostname to lower case. If hostname is null or "0.0.0.0", it uses
    * dynamically looked-up fqdn of the current host instead.
-   * 
+   * If domain_realm mappings are inadequately specified, it will
+   * use default_realm, per usual Kerberos behavior.
+   * If default_realm also gives a null value, then a principal
+   * without realm will be returned, which by Kerberos definitions is
+   * just another way to specify default realm.
+   *
    * @param service
    *          Service for which you want to generate the principal.
    * @param hostname
@@ -102,15 +164,26 @@ public class KerberosUtil {
    * @throws UnknownHostException
    *           If no IP address for the local host could be found.
    */
-  public static final String getServicePrincipal(String service, String hostname)
+  public static final String getServicePrincipal(String service,
+      String hostname)
       throws UnknownHostException {
     String fqdn = hostname;
+    String shortprinc = null;
+    String realmString = null;
     if (null == fqdn || fqdn.equals("") || fqdn.equals("0.0.0.0")) {
       fqdn = getLocalHostName();
     }
     // convert hostname to lowercase as kerberos does not work with hostnames
     // with uppercase characters.
-    return service + "/" + fqdn.toLowerCase(Locale.US);
+    fqdn = fqdn.toLowerCase(Locale.US);
+    shortprinc = service + "/" + fqdn;
+    // Obtain the realm name inferred from the domain of the host
+    realmString = getDomainRealm(shortprinc);
+    if (null == realmString || realmString.equals("")) {
+      return shortprinc;
+    } else {
+      return shortprinc + "@" + realmString;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ada9c2c4/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosUtil.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosUtil.java
index 89e07d1..a0ae025 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosUtil.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestKerberosUtil.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.security.authentication.util;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -52,32 +53,48 @@ public class TestKerberosUtil {
   }
 
   @Test
-  public void testGetServerPrincipal() throws IOException {
+  public void testGetServerPrincipal()
+      throws IOException, UnknownHostException {
     String service = "TestKerberosUtil";
     String localHostname = KerberosUtil.getLocalHostName();
     String testHost = "FooBar";
+    String defaultRealm = KerberosUtil.getDefaultRealmProtected();
+
+    String atDefaultRealm;
+    if (defaultRealm == null || defaultRealm.equals("")) {
+      atDefaultRealm = "";
+    } else {
+      atDefaultRealm = "@" + defaultRealm;
+    }
+    // check that the test environment is as expected
+    Assert.assertEquals("testGetServerPrincipal assumes localhost realm is default",
+        KerberosUtil.getDomainRealm(service + "/" + localHostname.toLowerCase(Locale.US)),
+        defaultRealm);
+    Assert.assertEquals("testGetServerPrincipal assumes realm of testHost 'FooBar' is default",
+        KerberosUtil.getDomainRealm(service + "/" + testHost.toLowerCase(Locale.US)),
+        defaultRealm);
 
     // send null hostname
     Assert.assertEquals("When no hostname is sent",
-        service + "/" + localHostname.toLowerCase(Locale.ENGLISH),
+        service + "/" + localHostname.toLowerCase(Locale.US) + atDefaultRealm,
         KerberosUtil.getServicePrincipal(service, null));
     // send empty hostname
     Assert.assertEquals("When empty hostname is sent",
-        service + "/" + localHostname.toLowerCase(Locale.ENGLISH),
+        service + "/" + localHostname.toLowerCase(Locale.US) + atDefaultRealm,
         KerberosUtil.getServicePrincipal(service, ""));
     // send 0.0.0.0 hostname
     Assert.assertEquals("When 0.0.0.0 hostname is sent",
-        service + "/" + localHostname.toLowerCase(Locale.ENGLISH),
+        service + "/" + localHostname.toLowerCase(Locale.US) + atDefaultRealm,
         KerberosUtil.getServicePrincipal(service, "0.0.0.0"));
     // send uppercase hostname
     Assert.assertEquals("When uppercase hostname is sent",
-        service + "/" + testHost.toLowerCase(Locale.ENGLISH),
+        service + "/" + testHost.toLowerCase(Locale.US) + atDefaultRealm,
         KerberosUtil.getServicePrincipal(service, testHost));
     // send lowercase hostname
     Assert.assertEquals("When lowercase hostname is sent",
-        service + "/" + testHost.toLowerCase(Locale.ENGLISH),
+        service + "/" + testHost.toLowerCase(Locale.US) + atDefaultRealm,
         KerberosUtil.getServicePrincipal(
-            service, testHost.toLowerCase(Locale.ENGLISH)));
+            service, testHost.toLowerCase(Locale.US)));
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ada9c2c4/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4cd295e..0f505dd 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -264,6 +264,9 @@ Trunk (Unreleased)
 
   BUG FIXES
 
+    HADOOP-12617. SPNEGO authentication request to non-default realm gets
+    default realm name inserted in target server principal. (mattf)
+
     HADOOP-11473. test-patch says "-1 overall" even when all checks are +1
     (Jason Lowe via raviprak)
 


[14/37] hadoop git commit: HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)

Posted by as...@apache.org.
HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/832b3cbd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/832b3cbd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/832b3cbd

Branch: refs/heads/yarn-2877
Commit: 832b3cbde1c2f77b04c93188e3a94420974090cf
Parents: 9f50e13
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Dec 8 12:56:38 2015 -0800
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue Dec 8 12:56:38 2015 -0800

----------------------------------------------------------------------
 .../fs/CommonConfigurationKeysPublic.java       |  3 +
 .../hadoop/security/UserGroupInformation.java   | 27 +++++++-
 .../src/main/resources/core-default.xml         |  6 ++
 .../security/TestUserGroupInformation.java      | 46 ++++++++++++-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 70 ++++++++++++++------
 .../hdfs/web/resources/DelegationParam.java     |  6 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../web/resources/NamenodeWebHdfsMethods.java   |  4 --
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      | 69 +++++++++++++++----
 .../apache/hadoop/hdfs/web/TestWebHdfsUrl.java  | 33 +++++++--
 10 files changed, 218 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index f75edd5..32dfc7e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -311,6 +311,9 @@ public class CommonConfigurationKeysPublic {
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_SECURITY_DNS_NAMESERVER_KEY =
     "hadoop.security.dns.nameserver";
+  /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
+  public static final String HADOOP_TOKEN_FILES =
+      "hadoop.token.files";
 
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 483420c..a9871a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.security;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
 import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
 
 import java.io.File;
@@ -248,9 +249,9 @@ public class UserGroupInformation {
   
   /**Environment variable pointing to the token cache file*/
   public static final String HADOOP_TOKEN_FILE_LOCATION = 
-    "HADOOP_TOKEN_FILE_LOCATION";
-  
-  /** 
+      "HADOOP_TOKEN_FILE_LOCATION";
+
+  /**
    * A method to initialize the fields that depend on a configuration.
    * Must be called before useKerberos or groups is used.
    */
@@ -821,6 +822,26 @@ public class UserGroupInformation {
       }
       loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
 
+      String tokenFileLocation = System.getProperty(HADOOP_TOKEN_FILES);
+      if (tokenFileLocation == null) {
+        tokenFileLocation = conf.get(HADOOP_TOKEN_FILES);
+      }
+      if (tokenFileLocation != null) {
+        String[] tokenFileNames = tokenFileLocation.split("\\s*,\\s*+");
+        for (String tokenFileName: tokenFileNames) {
+          if (tokenFileName.length() > 0) {
+            File tokenFile = new File(tokenFileName);
+            if (tokenFile.exists() && tokenFile.isFile()) {
+              Credentials cred = Credentials.readTokenStorageFile(
+                  tokenFile, conf);
+              loginUser.addCredentials(cred);
+            } else {
+              LOG.info("tokenFile("+tokenFileName+") does not exist");
+            }
+          }
+        }
+      }
+
       String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
       if (fileLocation != null) {
         // Load the token storage file and put all of the tokens into the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index c1a2946..318ab6f 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -466,6 +466,12 @@ for ldap providers in the same way as above does.
   <description>Maps kerberos principals to local user names</description>
 </property>
 
+<property>
+  <name>hadoop.token.files</name>
+  <value></value>
+  <description>List of token cache files that have delegation tokens for hadoop service</description>
+</property>
+
 <!-- i/o properties -->
 <property>
   <name>io.file.buffer.size</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index 54cfc2d..0539a03 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.security;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
@@ -35,6 +36,7 @@ import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.lang.reflect.Method;
@@ -861,7 +863,7 @@ public class TestUserGroupInformation {
 
     // Ensure only non-private tokens are returned
     Collection<Token<? extends TokenIdentifier>> tokens = ugi.getCredentials().getAllTokens();
-    assertEquals(1, tokens.size());
+    assertEquals(3, tokens.size());
   }
 
   /**
@@ -928,4 +930,46 @@ public class TestUserGroupInformation {
       }
     }
   }
+
+  @Test (timeout = 30000)
+  public void testExternalTokenFiles() throws Exception {
+    StringBuilder tokenFullPathnames = new StringBuilder();
+    String tokenFilenames = "token1,token2";
+    String tokenFiles[] = tokenFilenames.split("\\s*,\\s*+");
+    final File testDir = new File("target",
+        TestUserGroupInformation.class.getName() + "-tmpDir").getAbsoluteFile();
+    String testDirPath = testDir.getAbsolutePath();
+
+    // create path for token files
+    for (String tokenFile: tokenFiles) {
+      if (tokenFullPathnames.length() > 0) {
+        tokenFullPathnames.append(",");
+      }
+      tokenFullPathnames.append(testDirPath).append("/").append(tokenFile);
+    }
+
+    // create new token and store it
+    TestTokenIdentifier tokenId = new TestTokenIdentifier();
+    Credentials cred1 = new Credentials();
+    Token<TestTokenIdentifier> token1 = new Token<TestTokenIdentifier>(
+            tokenId.getBytes(), "password".getBytes(),
+            tokenId.getKind(), new Text("token-service1"));
+    cred1.addToken(token1.getService(), token1);
+    cred1.writeTokenStorageFile(new Path(testDirPath, tokenFiles[0]), conf);
+
+    Credentials cred2 = new Credentials();
+    Token<TestTokenIdentifier> token2 = new Token<TestTokenIdentifier>(
+            tokenId.getBytes(), "password".getBytes(),
+            tokenId.getKind(), new Text("token-service2"));
+    cred2.addToken(token2.getService(), token2);
+    cred2.writeTokenStorageFile(new Path(testDirPath, tokenFiles[1]), conf);
+
+    // set property for token external token files
+    System.setProperty("hadoop.token.files", tokenFullPathnames.toString());
+    UserGroupInformation.setLoginUser(null);
+    UserGroupInformation tokenUgi = UserGroupInformation.getLoginUser();
+    Collection<Token<?>> credsugiTokens = tokenUgi.getTokens();
+    assertTrue(credsugiTokens.contains(token1));
+    assertTrue(credsugiTokens.contains(token2));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 4049b80..c2a7ef8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -231,7 +231,7 @@ public class WebHdfsFileSystem extends FileSystem
   // the first getAuthParams() for a non-token op will either get the
   // internal token from the ugi or lazy fetch one
   protected synchronized Token<?> getDelegationToken() throws IOException {
-    if (canRefreshDelegationToken && delegationToken == null) {
+    if (delegationToken == null) {
       Token<?> token = tokenSelector.selectToken(
           new Text(getCanonicalServiceName()), ugi.getTokens());
       // ugi tokens are usually indicative of a task which can't
@@ -241,11 +241,13 @@ public class WebHdfsFileSystem extends FileSystem
         LOG.debug("Using UGI token: {}", token);
         canRefreshDelegationToken = false;
       } else {
-        token = getDelegationToken(null);
-        if (token != null) {
-          LOG.debug("Fetched new token: {}", token);
-        } else { // security is disabled
-          canRefreshDelegationToken = false;
+        if (canRefreshDelegationToken) {
+          token = getDelegationToken(null);
+          if (token != null) {
+            LOG.debug("Fetched new token: {}", token);
+          } else { // security is disabled
+            canRefreshDelegationToken = false;
+          }
         }
       }
       setDelegationToken(token);
@@ -257,6 +259,7 @@ public class WebHdfsFileSystem extends FileSystem
   synchronized boolean replaceExpiredDelegationToken() throws IOException {
     boolean replaced = false;
     if (canRefreshDelegationToken) {
+      this.delegationToken = null;
       Token<?> token = getDelegationToken(null);
       LOG.debug("Replaced expired token: {}", token);
       setDelegationToken(token);
@@ -1346,7 +1349,7 @@ public class WebHdfsFileSystem extends FileSystem
     final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS;
     return new FsPathResponseRunner<FileStatus[]>(op, f) {
       @Override
-      FileStatus[] decodeResponse(Map<?,?> json) {
+      FileStatus[] decodeResponse(Map<?, ?> json) {
         final Map<?, ?> rootmap =
             (Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
         final List<?> array = JsonUtilClient.getList(rootmap,
@@ -1367,18 +1370,34 @@ public class WebHdfsFileSystem extends FileSystem
   }
 
   @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(
+  public synchronized Token<DelegationTokenIdentifier> getDelegationToken(
       final String renewer) throws IOException {
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
-    Token<DelegationTokenIdentifier> token =
-        new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
-            op, null, new RenewerParam(renewer)) {
+    Token<DelegationTokenIdentifier> token = null;
+
+    if (delegationToken == null) {
+      token =
+          new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
+              op, null, new RenewerParam(renewer)) {
           @Override
-          Token<DelegationTokenIdentifier> decodeResponse(Map<?,?> json)
+          Token<DelegationTokenIdentifier> decodeResponse(Map<?, ?> json)
               throws IOException {
             return JsonUtilClient.toDelegationToken(json);
           }
         }.run();
+    } else {
+      token =
+          new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
+              op, null, new RenewerParam(renewer),
+              new DelegationParam(delegationToken.encodeToUrlString())) {
+          @Override
+          Token<DelegationTokenIdentifier> decodeResponse(Map<?, ?> json)
+              throws IOException {
+            return JsonUtilClient.toDelegationToken(json);
+          }
+        }.run();
+    }
+
     if (token != null) {
       token.setService(tokenServiceName);
     } else {
@@ -1406,13 +1425,26 @@ public class WebHdfsFileSystem extends FileSystem
   public synchronized long renewDelegationToken(final Token<?> token
   ) throws IOException {
     final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
-    return new FsPathResponseRunner<Long>(op, null,
-        new TokenArgumentParam(token.encodeToUrlString())) {
-      @Override
-      Long decodeResponse(Map<?,?> json) throws IOException {
-        return ((Number) json.get("long")).longValue();
-      }
-    }.run();
+
+    if (delegationToken == null) {
+      return new FsPathResponseRunner<Long>(op, null,
+          new TokenArgumentParam(token.encodeToUrlString())) {
+        @Override
+        Long decodeResponse(Map<?, ?> json) throws IOException {
+          return ((Number) json.get("long")).longValue();
+        }
+      }.run();
+    } else {
+      return new FsPathResponseRunner<Long>(op, null,
+          new TokenArgumentParam(token.encodeToUrlString()),
+          new DelegationParam(delegationToken.encodeToUrlString())) {
+        @Override
+        Long decodeResponse(Map<?, ?> json) throws IOException {
+          return ((Number) json.get("long")).longValue();
+        }
+      }.run();
+    }
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
index 5329580..fb129d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.web.resources;
 
-import org.apache.hadoop.security.UserGroupInformation;
-
 /** Represents delegation token used for authentication. */
 public class DelegationParam extends StringParam {
   /** Parameter name. */
@@ -33,8 +31,8 @@ public class DelegationParam extends StringParam {
    * @param str a string representation of the parameter value.
    */
   public DelegationParam(final String str) {
-    super(DOMAIN, UserGroupInformation.isSecurityEnabled()
-        && str != null && !str.equals(DEFAULT)? str: null);
+    super(DOMAIN, str != null && !str.equals(DEFAULT)? str: null);
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d3e7e02..8a892d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -55,6 +55,9 @@ Trunk (Unreleased)
     HDFS-9057. allow/disallow snapshots via webhdfs
     (Bramma Reddy Battula via vinayakumarb)
 
+    HDFS-9525. hadoop utilities need to support provided delegation tokens
+    (HeeSoo Kim via aw)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 4626507..5e602b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -895,10 +895,6 @@ public class NamenodeWebHdfsMethods {
     }
     case GETDELEGATIONTOKEN:
     {
-      if (delegation.getValue() != null) {
-        throw new IllegalArgumentException(delegation.getName()
-            + " parameter is not null.");
-      }
       final Token<? extends TokenIdentifier> token = generateDelegationToken(
           namenode, ugi, renewer.getValue());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
index 5e8568c..b17cb4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
@@ -296,7 +296,59 @@ public class TestWebHdfsTokens {
       }
     }
   }
-  
+
+  @Test
+  public void testReuseToken() throws Exception {
+    MiniDFSCluster cluster = null;
+
+    UserGroupInformation loginUgi = UserGroupInformation.createUserForTesting(
+        "LoginUser", new String[]{"supergroup"});
+
+    try {
+      final Configuration clusterConf = new HdfsConfiguration(conf);
+      SecurityUtil.setAuthenticationMethod(SIMPLE, clusterConf);
+      clusterConf.setBoolean(DFSConfigKeys
+          .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+      UserGroupInformation.setConfiguration(clusterConf);
+      UserGroupInformation.setLoginUser(loginUgi);
+
+      cluster = new MiniDFSCluster.Builder(clusterConf).numDataNodes(0).build();
+      cluster.waitActive();
+
+      /* create SIMPLE client connection */
+      SecurityUtil.setAuthenticationMethod(SIMPLE, clusterConf);
+      UserGroupInformation.setConfiguration(clusterConf);
+      UserGroupInformation simpleUgi = UserGroupInformation.createUserForTesting(
+          "testUser", new String[]{"supergroup"});
+      final WebHdfsFileSystem simpleFs = WebHdfsTestUtil.getWebHdfsFileSystemAs
+          (simpleUgi, clusterConf, "webhdfs");
+
+      /* create KERBEROS client connection */
+      SecurityUtil.setAuthenticationMethod(KERBEROS, clusterConf);
+      UserGroupInformation.setConfiguration(clusterConf);
+      UserGroupInformation krbUgi = UserGroupInformation.createUserForTesting(
+          "testUser", new String[]{"supergroup"});
+      final WebHdfsFileSystem krbFs = WebHdfsTestUtil.getWebHdfsFileSystemAs
+              (krbUgi, clusterConf, "webhdfs");
+
+      // 1. Get initial token through kerberos client connection
+      Token<DelegationTokenIdentifier> krbToken
+        = krbFs.getDelegationToken(null);
+      Assert.assertNotNull(krbToken);
+
+      // 2. Get token with previous token which gets from kerberos connection
+      //    through SIMPLE client connection.
+      simpleFs.setDelegationToken(krbToken);
+      Token<?> simpleToken =  simpleFs.getDelegationToken();
+      Assert.assertNotNull(simpleToken);
+      Assert.assertEquals(krbToken.getService(), simpleToken.getService());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   @SuppressWarnings("unchecked")
   private void validateLazyTokenFetch(final Configuration clusterConf) throws Exception{
     final String testUser = "DummyUser";
@@ -308,16 +360,6 @@ public class TestWebHdfsTokens {
         return spy((WebHdfsFileSystem) FileSystem.newInstance(uri, clusterConf));
 	  }
     });
-    // verify token ops don't get a token
-    Assert.assertNull(fs.getRenewToken());
-    Token<?> token = fs.getDelegationToken(null);
-    fs.renewDelegationToken(token);
-    fs.cancelDelegationToken(token);
-    verify(fs, never()).getDelegationToken();
-    verify(fs, never()).replaceExpiredDelegationToken();
-    verify(fs, never()).setDelegationToken(any(Token.class));
-    Assert.assertNull(fs.getRenewToken());
-    reset(fs);
 
     // verify first non-token op gets a token
     final Path p = new Path("/f");
@@ -326,8 +368,8 @@ public class TestWebHdfsTokens {
     verify(fs, never()).replaceExpiredDelegationToken();
     verify(fs, times(1)).getDelegationToken(anyString());
     verify(fs, times(1)).setDelegationToken(any(Token.class));
-    token = fs.getRenewToken();
-    Assert.assertNotNull(token);      
+    Token<?> token = fs.getRenewToken();
+    Assert.assertNotNull(token);
     Assert.assertEquals(testUser, getTokenOwner(token));
     Assert.assertEquals(fs.getTokenKind(), token.getKind());
     reset(fs);
@@ -421,6 +463,7 @@ public class TestWebHdfsTokens {
     verify(fs, times(1)).cancelDelegationToken(eq(token2));
 
     // add a token to ugi for a new fs, verify it uses that token
+    fs.setDelegationToken(null);
     token = fs.getDelegationToken(null);
     ugi.addToken(token);
     fs = ugi.doAs(new PrivilegedExceptionAction<WebHdfsFileSystem>() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/832b3cbd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
index 2913a97..ba1c8a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
@@ -195,9 +195,33 @@ public class TestWebHdfsUrl {
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString()
+            new DelegationParam(tokenString).toString()
         },
-        fileStatusUrl);    
+        fileStatusUrl);
+
+    // send user with delegationToken
+    getTokenUrl = webhdfs.toUrl(GetOpParam.Op.GETDELEGATIONTOKEN,
+        fsPath, new DelegationParam(tokenString));
+    checkQueryParams(
+        new String[]{
+            GetOpParam.Op.GETDELEGATIONTOKEN.toQueryString(),
+            new UserParam(ugi.getShortUserName()).toString(),
+            new DelegationParam(tokenString).toString()
+        },
+        getTokenUrl);
+
+    // send user with delegationToken
+    renewTokenUrl = webhdfs.toUrl(PutOpParam.Op.RENEWDELEGATIONTOKEN,
+        fsPath, new TokenArgumentParam(tokenString),
+        new DelegationParam(tokenString));
+    checkQueryParams(
+        new String[]{
+            PutOpParam.Op.RENEWDELEGATIONTOKEN.toQueryString(),
+            new UserParam(ugi.getShortUserName()).toString(),
+            new TokenArgumentParam(tokenString).toString(),
+            new DelegationParam(tokenString).toString()
+        },
+        renewTokenUrl);
   }
 
   @Test(timeout=60000)
@@ -274,14 +298,13 @@ public class TestWebHdfsUrl {
             new TokenArgumentParam(tokenString).toString()
         },
         cancelTokenUrl);
-    
+
     // send real+effective
     fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new UserParam(ugi.getRealUser().getShortUserName()).toString(),
-            new DoAsParam(ugi.getShortUserName()).toString()
+            new DelegationParam(tokenString).toString()
         },
         fileStatusUrl);    
   }


[27/37] hadoop git commit: HADOOP-12602. TestMetricsSystemImpl#testQSize occasionally fails. Contributed by Masatake Iwasaki.

Posted by as...@apache.org.
HADOOP-12602. TestMetricsSystemImpl#testQSize occasionally fails. Contributed by Masatake Iwasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/eee0cf46
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eee0cf46
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eee0cf46

Branch: refs/heads/yarn-2877
Commit: eee0cf4611b02171e8a043f1cc5f7dbad47fc3b4
Parents: 7f393a6
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Dec 11 16:18:30 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Dec 11 16:18:30 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java    | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eee0cf46/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 61ef7ba..93766f9 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1540,6 +1540,9 @@ Release 2.7.3 - UNRELEASED
     HADOOP-12565. Replace DSA with RSA for SSH key type in SingleCluster.md.
     (Mingliang Liu via aajisaka)
 
+    HADOOP-12602. TestMetricsSystemImpl#testQSize occasionally fails.
+    (Masatake Iwasaki via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eee0cf46/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
index ecf2cc8..0463f48 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
@@ -441,6 +441,7 @@ public class TestMetricsSystemImpl {
   @Test
   public void testQSize() throws Exception {
     new ConfigBuilder().add("*.period", 8)
+        .add("*.queue.capacity", 2)
         .add("test.sink.test.class", TestSink.class.getName())
         .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
     MetricsSystemImpl ms = new MetricsSystemImpl("Test");


[24/37] hadoop git commit: YARN-3623. Add a config to indicate the Timeline Service version. Contributed by Xuan Gong.

Posted by as...@apache.org.
YARN-3623. Add a config to indicate the Timeline Service version. Contributed by Xuan Gong.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f910e4f6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f910e4f6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f910e4f6

Branch: refs/heads/yarn-2877
Commit: f910e4f639dc311fcb257bfcb869b1aa8b2c0643
Parents: 132478e
Author: Junping Du <ju...@apache.org>
Authored: Thu Dec 10 02:44:30 2015 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Thu Dec 10 02:53:47 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                       |  3 +++
 .../apache/hadoop/yarn/conf/YarnConfiguration.java    |  3 +++
 .../src/main/resources/yarn-default.xml               | 14 ++++++++++++++
 3 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f910e4f6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 29b9570..32d76f4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -271,6 +271,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4349. Support CallerContext in YARN. (wtan via jianhe)
 
+    YARN-3623. Add a new config to indicate the Timeline Service version. 
+    (Xuan Gong via junping_du)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f910e4f6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index cbd28ca..2d9ca72 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1538,6 +1538,9 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_SERVICE_PREFIX =
       YARN_PREFIX + "timeline-service.";
 
+  public static final String TIMELINE_SERVICE_VERSION = TIMELINE_SERVICE_PREFIX
+      + "version";
+  public static final float DEFAULT_TIMELINE_SERVICE_VERSION = 1.0f;
   /**
    * Comma seperated list of names for UIs hosted in the timeline server
    * (For pluggable UIs).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f910e4f6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index c862ef2..0917366 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1766,6 +1766,20 @@
   <!-- Timeline Service Configuration -->
 
   <property>
+    <description>Indicate what is the current version of the running
+    timeline service. For example, if "yarn.timeline-service.version" is 1.5,
+    and "yarn.timeline-service.enabled" is true, it means the cluster will and
+    should bring up the timeline service v.1.5 (and nothing else).
+    On the client side, if the client uses the same version of timeline service,
+    it should succeed. If the client chooses to use a smaller version in spite of this,
+    then depending on how robust the compatibility story is between versions,
+    the results may vary.
+    </description>
+    <name>yarn.timeline-service.version</name>
+    <value>1.0f</value>
+  </property>
+
+  <property>
     <description>Indicate to clients whether timeline service is enabled or not.
     If enabled, clients will put entities and events to the timeline server.
     </description>


[28/37] hadoop git commit: HDFS-9472. concat() API does not give proper exception messages on ./reserved relative path (Rakesh R via umamahesh)

Posted by as...@apache.org.
HDFS-9472. concat() API does not give proper exception messages on ./reserved relative path (Rakesh R via umamahesh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e363417e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e363417e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e363417e

Branch: refs/heads/yarn-2877
Commit: e363417e7b7abdd5d149f303f729ecf3e95ef8f3
Parents: eee0cf4
Author: Uma Mahesh <um...@apache.org>
Authored: Thu Dec 10 23:55:29 2015 -0800
Committer: Uma Mahesh <um...@apache.org>
Committed: Thu Dec 10 23:55:29 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hdfs/server/namenode/FSDirConcatOp.java     | 24 ++++++++++++++++----
 .../hdfs/server/namenode/TestHDFSConcat.java    | 19 ++++++++++++++++
 3 files changed, 42 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e363417e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7fe5850..1696053 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1737,6 +1737,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9527. The return type of FSNamesystem.getBlockCollection should be
     changed to INodeFile. (szetszwo)
 
+    HDFS-9472. concat() API does not give proper exception messages on ./reserved 
+    relative path (Rakesh R via umamahesh)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e363417e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
index 83782d5..5310b94 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
@@ -49,14 +49,11 @@ class FSDirConcatOp {
 
   static HdfsFileStatus concat(FSDirectory fsd, String target, String[] srcs,
     boolean logRetryCache) throws IOException {
-    Preconditions.checkArgument(!target.isEmpty(), "Target file name is empty");
-    Preconditions.checkArgument(srcs != null && srcs.length > 0,
-      "No sources given");
+    validatePath(target, srcs);
     assert srcs != null;
     if (FSDirectory.LOG.isDebugEnabled()) {
       FSDirectory.LOG.debug("concat {} to {}", Arrays.toString(srcs), target);
     }
-
     final INodesInPath targetIIP = fsd.getINodesInPath4Write(target);
     // write permission for the target
     FSPermissionChecker pc = null;
@@ -86,6 +83,25 @@ class FSDirConcatOp {
     return fsd.getAuditFileInfo(targetIIP);
   }
 
+  private static void validatePath(String target, String[] srcs)
+      throws IOException {
+    Preconditions.checkArgument(!target.isEmpty(), "Target file name is empty");
+    Preconditions.checkArgument(srcs != null && srcs.length > 0,
+        "No sources given");
+    if (FSDirectory.isReservedRawName(target)
+        || FSDirectory.isReservedInodesName(target)) {
+      throw new IOException("Concat operation doesn't support "
+          + FSDirectory.DOT_RESERVED_STRING + " relative path : " + target);
+    }
+    for (String srcPath : srcs) {
+      if (FSDirectory.isReservedRawName(srcPath)
+          || FSDirectory.isReservedInodesName(srcPath)) {
+        throw new IOException("Concat operation doesn't support "
+            + FSDirectory.DOT_RESERVED_STRING + " relative path : " + srcPath);
+      }
+    }
+  }
+
   private static void verifyTargetFile(FSDirectory fsd, final String target,
       final INodesInPath targetIIP) throws IOException {
     // check the target

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e363417e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
index ded0031..6dc0782 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
@@ -503,4 +503,23 @@ public class TestHDFSConcat {
     assertEquals(blockSize * 2, dfs.getFileStatus(trg).getLen());
     assertFalse(dfs.exists(src));
   }
+
+  @Test(timeout = 30000)
+  public void testConcatReservedRelativePaths() throws IOException {
+    String testPathDir = "/.reserved/raw/ezone";
+    Path dir = new Path(testPathDir);
+    dfs.mkdirs(dir);
+    Path trg = new Path(testPathDir, "trg");
+    Path src = new Path(testPathDir, "src");
+    DFSTestUtil.createFile(dfs, trg, blockSize, REPL_FACTOR, 1);
+    DFSTestUtil.createFile(dfs, src, blockSize, REPL_FACTOR, 1);
+    try {
+      dfs.concat(trg, new Path[] { src });
+      Assert.fail("Must throw Exception!");
+    } catch (IOException e) {
+      String errMsg = "Concat operation doesn't support "
+          + FSDirectory.DOT_RESERVED_STRING + " relative path : " + trg;
+      GenericTestUtils.assertExceptionContains(errMsg, e);
+    }
+  }
 }


[07/37] hadoop git commit: MAPREDUCE-6566. Add retry support to mapreduce CLI tool. Contributed by Varun Vasudev

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc470840/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
index 59f5ada..db87d9d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.HashSet;
 import java.util.Arrays;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -43,6 +44,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskCompletionEvent;
 import org.apache.hadoop.mapreduce.TaskReport;
@@ -268,7 +270,7 @@ public class CLI extends Configured implements Tool {
         System.out.println("Created job " + job.getJobID());
         exitCode = 0;
       } else if (getStatus) {
-        Job job = cluster.getJob(JobID.forName(jobid));
+        Job job = getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -283,7 +285,7 @@ public class CLI extends Configured implements Tool {
           exitCode = 0;
         }
       } else if (getCounter) {
-        Job job = cluster.getJob(JobID.forName(jobid));
+        Job job = getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -299,7 +301,7 @@ public class CLI extends Configured implements Tool {
           }
         }
       } else if (killJob) {
-        Job job = cluster.getJob(JobID.forName(jobid));
+        Job job = getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -323,7 +325,7 @@ public class CLI extends Configured implements Tool {
           }
         }
       } else if (setJobPriority) {
-        Job job = cluster.getJob(JobID.forName(jobid));
+        Job job = getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -339,7 +341,7 @@ public class CLI extends Configured implements Tool {
         viewHistory(historyFile, viewAllHistory);
         exitCode = 0;
       } else if (listEvents) {
-        listEvents(cluster.getJob(JobID.forName(jobid)), fromEvent, nEvents);
+        listEvents(getJob(JobID.forName(jobid)), fromEvent, nEvents);
         exitCode = 0;
       } else if (listJobs) {
         listJobs(cluster);
@@ -354,11 +356,11 @@ public class CLI extends Configured implements Tool {
         listBlacklistedTrackers(cluster);
         exitCode = 0;
       } else if (displayTasks) {
-        displayTasks(cluster.getJob(JobID.forName(jobid)), taskType, taskState);
+        displayTasks(getJob(JobID.forName(jobid)), taskType, taskState);
         exitCode = 0;
       } else if(killTask) {
         TaskAttemptID taskID = TaskAttemptID.forName(taskid);
-        Job job = cluster.getJob(taskID.getJobID());
+        Job job = getJob(taskID.getJobID());
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else if (job.killTask(taskID, false)) {
@@ -370,7 +372,7 @@ public class CLI extends Configured implements Tool {
         }
       } else if(failTask) {
         TaskAttemptID taskID = TaskAttemptID.forName(taskid);
-        Job job = cluster.getJob(taskID.getJobID());
+        Job job = getJob(taskID.getJobID());
         if (job == null) {
             System.out.println("Could not find job " + jobid);
         } else if(job.killTask(taskID, true)) {
@@ -531,6 +533,29 @@ public class CLI extends Configured implements Tool {
   protected static String getTaskLogURL(TaskAttemptID taskId, String baseUrl) {
     return (baseUrl + "/tasklog?plaintext=true&attemptid=" + taskId); 
   }
+
+  @VisibleForTesting
+  Job getJob(JobID jobid) throws IOException, InterruptedException {
+
+    int maxRetry = getConf().getInt(MRJobConfig.MR_CLIENT_JOB_MAX_RETRIES,
+        MRJobConfig.DEFAULT_MR_CLIENT_JOB_MAX_RETRIES);
+    long retryInterval = getConf()
+        .getLong(MRJobConfig.MR_CLIENT_JOB_RETRY_INTERVAL,
+            MRJobConfig.DEFAULT_MR_CLIENT_JOB_RETRY_INTERVAL);
+    Job job = cluster.getJob(jobid);
+
+    for (int i = 0; i < maxRetry; ++i) {
+      if (job != null) {
+        return job;
+      }
+      LOG.info("Could not obtain job info after " + String.valueOf(i + 1)
+          + " attempt(s). Sleeping for " + String.valueOf(retryInterval / 1000)
+          + " seconds and retrying.");
+      Thread.sleep(retryInterval);
+      job = cluster.getJob(jobid);
+    }
+    return job;
+  }
   
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc470840/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java
index fdc916e..73f57d5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java
@@ -20,14 +20,19 @@ package org.apache.hadoop.mapreduce.tools;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Cluster;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskReport;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.JobStatus.State;
+import org.apache.hadoop.util.Time;
+import org.junit.Assert;
 import org.junit.Test;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -44,7 +49,7 @@ public class TestCLI {
     JobID jobId = JobID.forName(jobIdStr);
     Cluster mockCluster = mock(Cluster.class);
     Job job = mock(Job.class);
-    CLI cli = spy(new CLI());
+    CLI cli = spy(new CLI(new Configuration()));
 
     doReturn(mockCluster).when(cli).createCluster();
     when(job.getTaskReports(TaskType.MAP)).thenReturn(
@@ -112,7 +117,7 @@ public class TestCLI {
   @Test
   public void testJobKIll() throws Exception {
     Cluster mockCluster = mock(Cluster.class);
-    CLI cli = spy(new CLI());
+    CLI cli = spy(new CLI(new Configuration()));
     doReturn(mockCluster).when(cli).createCluster();
     String jobId1 = "job_1234654654_001";
     String jobId2 = "job_1234654654_002";
@@ -149,4 +154,26 @@ public class TestCLI {
     when(mockJob.getStatus()).thenReturn(status);
     return mockJob;
   }
+
+  @Test
+  public void testGetJob() throws Exception {
+    Configuration conf = new Configuration();
+    long sleepTime = 100;
+    conf.setLong(MRJobConfig.MR_CLIENT_JOB_RETRY_INTERVAL, sleepTime);
+    Cluster mockCluster = mock(Cluster.class);
+    JobID jobId1 = JobID.forName("job_1234654654_001");
+    when(mockCluster.getJob(jobId1)).thenReturn(null);
+
+    for (int i = 0; i < 2; ++i) {
+      conf.setInt(MRJobConfig.MR_CLIENT_JOB_MAX_RETRIES, i);
+      CLI cli = spy(new CLI(conf));
+      cli.cluster = mockCluster;
+      doReturn(mockCluster).when(cli).createCluster();
+      long start = Time.monotonicNow();
+      cli.getJob(jobId1);
+      long end = Time.monotonicNow();
+      Assert.assertTrue(end - start > (i * sleepTime));
+      Assert.assertTrue(end - start < ((i + 1) * sleepTime));
+    }
+  }
 }


[26/37] hadoop git commit: Revert "HDFS-8860. Remove unused Replica copyOnWrite code (Lei (Eddy) Xu via Colin P. McCabe)"

Posted by as...@apache.org.
Revert "HDFS-8860. Remove unused Replica copyOnWrite code (Lei (Eddy) Xu via Colin P. McCabe)"

This reverts commit a153b9601ad8628fdd608d8696310ca8c1f58ff0.

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7f393a6f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7f393a6f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7f393a6f

Branch: refs/heads/yarn-2877
Commit: 7f393a6f61f5a34a1de11481ad321c6a941d5d27
Parents: 21daa6c
Author: Lei Xu <le...@apache.org>
Authored: Thu Dec 10 10:57:33 2015 -1000
Committer: Lei Xu <le...@apache.org>
Committed: Thu Dec 10 10:57:33 2015 -1000

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 -
 .../hdfs/server/datanode/FinalizedReplica.java  | 15 +++-
 .../hdfs/server/datanode/ReplicaInfo.java       | 88 ++++++++++++++++++++
 .../server/datanode/ReplicaUnderRecovery.java   | 10 +++
 .../datanode/ReplicaWaitingToBeRecovered.java   | 15 +++-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  3 +
 .../org/apache/hadoop/hdfs/TestFileAppend.java  | 72 ++++++++++++++++
 .../hdfs/server/datanode/DataNodeTestUtils.java | 15 ++++
 .../fsdataset/impl/FsDatasetTestUtil.java       |  6 ++
 .../fsdataset/impl/TestDatanodeRestart.java     | 72 ++++++++++++++++
 10 files changed, 294 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6755f00..7fe5850 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1448,8 +1448,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-9019. Adding informative message to sticky bit permission denied
     exception. (xyao)
 
-    HDFS-8860. Remove unused Replica copyOnWrite code (Lei (Eddy) Xu via Colin P. McCabe)
-
     HDFS-8716. Introduce a new config specifically for safe mode block count
     (Chang Li via kihwal)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
index 8daeb51..cc32874 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
  * This class describes a replica that has been finalized.
  */
 public class FinalizedReplica extends ReplicaInfo {
+  private boolean unlinked;      // copy-on-write done for block
 
   /**
    * Constructor
@@ -57,6 +58,7 @@ public class FinalizedReplica extends ReplicaInfo {
    */
   public FinalizedReplica(FinalizedReplica from) {
     super(from);
+    this.unlinked = from.isUnlinked();
   }
 
   @Override  // ReplicaInfo
@@ -64,6 +66,16 @@ public class FinalizedReplica extends ReplicaInfo {
     return ReplicaState.FINALIZED;
   }
   
+  @Override // ReplicaInfo
+  public boolean isUnlinked() {
+    return unlinked;
+  }
+
+  @Override  // ReplicaInfo
+  public void setUnlinked() {
+    unlinked = true;
+  }
+  
   @Override
   public long getVisibleLength() {
     return getNumBytes();       // all bytes are visible
@@ -86,6 +98,7 @@ public class FinalizedReplica extends ReplicaInfo {
   
   @Override
   public String toString() {
-    return super.toString();
+    return super.toString()
+        + "\n  unlinked          =" + unlinked;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
index d19e656..e41cce0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
@@ -18,12 +18,18 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.LightWeightResizableGSet;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -194,6 +200,22 @@ abstract public class ReplicaInfo extends Block
   }
 
   /**
+   * check if this replica has already been unlinked.
+   * @return true if the replica has already been unlinked 
+   *         or no need to be detached; false otherwise
+   */
+  public boolean isUnlinked() {
+    return true;                // no need to be unlinked
+  }
+
+  /**
+   * set that this replica is unlinked
+   */
+  public void setUnlinked() {
+    // no need to be unlinked
+  }
+
+  /**
    * Number of bytes reserved for this replica on disk.
    */
   public long getBytesReserved() {
@@ -210,6 +232,72 @@ abstract public class ReplicaInfo extends Block
     return 0;
   }
 
+   /**
+   * Copy specified file into a temporary file. Then rename the
+   * temporary file to the original name. This will cause any
+   * hardlinks to the original file to be removed. The temporary
+   * files are created in the same directory. The temporary files will
+   * be recovered (especially on Windows) on datanode restart.
+   */
+  private void unlinkFile(File file, Block b) throws IOException {
+    File tmpFile = DatanodeUtil.createTmpFile(b, DatanodeUtil.getUnlinkTmpFile(file));
+    try {
+      FileInputStream in = new FileInputStream(file);
+      try {
+        FileOutputStream out = new FileOutputStream(tmpFile);
+        try {
+          IOUtils.copyBytes(in, out, 16*1024);
+        } finally {
+          out.close();
+        }
+      } finally {
+        in.close();
+      }
+      if (file.length() != tmpFile.length()) {
+        throw new IOException("Copy of file " + file + " size " + file.length()+
+                              " into file " + tmpFile +
+                              " resulted in a size of " + tmpFile.length());
+      }
+      FileUtil.replaceFile(tmpFile, file);
+    } catch (IOException e) {
+      boolean done = tmpFile.delete();
+      if (!done) {
+        DataNode.LOG.info("detachFile failed to delete temporary file " +
+                          tmpFile);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Remove a hard link by copying the block to a temporary place and 
+   * then moving it back
+   * @param numLinks number of hard links
+   * @return true if copy is successful; 
+   *         false if it is already detached or no need to be detached
+   * @throws IOException if there is any copy error
+   */
+  public boolean unlinkBlock(int numLinks) throws IOException {
+    if (isUnlinked()) {
+      return false;
+    }
+    File file = getBlockFile();
+    if (file == null || getVolume() == null) {
+      throw new IOException("detachBlock:Block not found. " + this);
+    }
+    File meta = getMetaFile();
+
+    if (HardLink.getLinkCount(file) > numLinks) {
+      DataNode.LOG.info("CopyOnWrite for block " + this);
+      unlinkFile(file, this);
+    }
+    if (HardLink.getLinkCount(meta) > numLinks) {
+      unlinkFile(meta, this);
+    }
+    setUnlinked();
+    return true;
+  }
+
   @Override  //Object
   public String toString() {
     return getClass().getSimpleName()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
index 558ee21..2cd8a01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
@@ -85,6 +85,16 @@ public class ReplicaUnderRecovery extends ReplicaInfo {
   public ReplicaInfo getOriginalReplica() {
     return original;
   }
+
+  @Override //ReplicaInfo
+  public boolean isUnlinked() {
+    return original.isUnlinked();
+  }
+
+  @Override //ReplicaInfo
+  public void setUnlinked() {
+    original.setUnlinked();
+  }
   
   @Override //ReplicaInfo
   public ReplicaState getState() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
index 220649d..26ab3db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
  * lease recovery.
  */
 public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
+  private boolean unlinked;      // copy-on-write done for block
 
   /**
    * Constructor
@@ -63,6 +64,7 @@ public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
    */
   public ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
     super(from);
+    this.unlinked = from.isUnlinked();
   }
 
   @Override //ReplicaInfo
@@ -71,6 +73,16 @@ public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
   }
   
   @Override //ReplicaInfo
+  public boolean isUnlinked() {
+    return unlinked;
+  }
+
+  @Override //ReplicaInfo
+  public void setUnlinked() {
+    unlinked = true;
+  }
+  
+  @Override //ReplicaInfo
   public long getVisibleLength() {
     return -1;  //no bytes are visible
   }
@@ -92,6 +104,7 @@ public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
 
   @Override
   public String toString() {
-    return super.toString();
+    return super.toString()
+        + "\n  unlinked=" + unlinked;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 466c7e9..1d8c705 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1109,6 +1109,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       throws IOException {
     // If the block is cached, start uncaching it.
     cacheManager.uncacheBlock(bpid, replicaInfo.getBlockId());
+    // unlink the finalized replica
+    replicaInfo.unlinkBlock(1);
     
     // construct a RBW replica with the new GS
     File blkfile = replicaInfo.getBlockFile();
@@ -2478,6 +2480,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           + ", rur=" + rur);
     }
     if (rur.getNumBytes() > newlength) {
+      rur.unlinkBlock(1);
       truncateBlock(blockFile, metaFile, rur.getNumBytes(), newlength);
       if(!copyOnTruncate) {
         // update RUR with the new length

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index 7b7f415..85d92c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -110,6 +110,78 @@ public class TestFileAppend{
   }
 
   /**
+   * Test that copy on write for blocks works correctly
+   * @throws IOException an exception might be thrown
+   */
+  @Test
+  public void testCopyOnWrite() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    if (simulatedStorage) {
+      SimulatedFSDataset.setFactory(conf);
+    }
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    FileSystem fs = cluster.getFileSystem();
+    InetSocketAddress addr = new InetSocketAddress("localhost",
+                                                   cluster.getNameNodePort());
+    DFSClient client = new DFSClient(addr, conf);
+    try {
+
+      // create a new file, write to it and close it.
+      //
+      Path file1 = new Path("/filestatus.dat");
+      FSDataOutputStream stm = AppendTestUtil.createFile(fs, file1, 1);
+      writeFile(stm);
+      stm.close();
+
+      // Get a handle to the datanode
+      DataNode[] dn = cluster.listDataNodes();
+      assertTrue("There should be only one datanode but found " + dn.length,
+                  dn.length == 1);
+
+      LocatedBlocks locations = client.getNamenode().getBlockLocations(
+                                  file1.toString(), 0, Long.MAX_VALUE);
+      List<LocatedBlock> blocks = locations.getLocatedBlocks();
+
+      //
+      // Create hard links for a few of the blocks
+      //
+      for (int i = 0; i < blocks.size(); i = i + 2) {
+        ExtendedBlock b = blocks.get(i).getBlock();
+        final File f = DataNodeTestUtils.getFile(dn[0],
+            b.getBlockPoolId(), b.getLocalBlock().getBlockId());
+        File link = new File(f.toString() + ".link");
+        System.out.println("Creating hardlink for File " + f + " to " + link);
+        HardLink.createHardLink(f, link);
+      }
+
+      //
+      // Detach all blocks. This should remove hardlinks (if any)
+      //
+      for (int i = 0; i < blocks.size(); i++) {
+        ExtendedBlock b = blocks.get(i).getBlock();
+        System.out.println("testCopyOnWrite detaching block " + b);
+        assertTrue("Detaching block " + b + " should have returned true",
+            DataNodeTestUtils.unlinkBlock(dn[0], b, 1));
+      }
+
+      // Since the blocks were already detached earlier, these calls should
+      // return false
+      //
+      for (int i = 0; i < blocks.size(); i++) {
+        ExtendedBlock b = blocks.get(i).getBlock();
+        System.out.println("testCopyOnWrite detaching block " + b);
+        assertTrue("Detaching block " + b + " should have returned false",
+            !DataNodeTestUtils.unlinkBlock(dn[0], b, 1));
+      }
+
+    } finally {
+      client.close();
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+
+  /**
    * Test a simple flush on a simple HDFS file.
    * @throws IOException an exception might be thrown
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
index 1d47192..6bcbb1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
@@ -158,6 +159,20 @@ public class DataNodeTestUtils {
     return dn.getFSDataset();
   }
 
+  public static File getFile(DataNode dn, String bpid, long bid) {
+    return FsDatasetTestUtil.getFile(dn.getFSDataset(), bpid, bid);
+  }
+
+  public static File getBlockFile(DataNode dn, String bpid, Block b
+      ) throws IOException {
+    return FsDatasetTestUtil.getBlockFile(dn.getFSDataset(), bpid, b);
+  }
+  
+  public static boolean unlinkBlock(DataNode dn, ExtendedBlock bk, int numLinks
+      ) throws IOException {
+    return FsDatasetTestUtil.unlinkBlock(dn.getFSDataset(), bk, numLinks);
+  }
+
   /**
    * Fetch a copy of ReplicaInfo from a datanode by block id
    * @param dn datanode to retrieve a replicainfo object from

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
index f4480a1..9c297e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
@@ -54,6 +54,12 @@ public class FsDatasetTestUtil {
     return FsDatasetUtil.getMetaFile(getBlockFile(fsd, bpid, b), b
         .getGenerationStamp());
   }
+  
+  public static boolean unlinkBlock(FsDatasetSpi<?> fsd,
+      ExtendedBlock block, int numLinks) throws IOException {
+    final ReplicaInfo info = ((FsDatasetImpl)fsd).getReplicaInfo(block);
+    return info.unlinkBlock(numLinks);
+  }
 
   public static ReplicaInfo fetchReplicaInfo (final FsDatasetSpi<?> fsd,
       final String bpid, final long blockId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f393a6f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
index 8bbac9f..4516696 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java
@@ -143,7 +143,79 @@ public class TestDatanodeRestart {
     }      
   }
 
+  // test recovering unlinked tmp replicas
+  @Test public void testRecoverReplicas() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
+    conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitActive();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      for (int i=0; i<4; i++) {
+        Path fileName = new Path("/test"+i);
+        DFSTestUtil.createFile(fs, fileName, 1, (short)1, 0L);
+        DFSTestUtil.waitReplication(fs, fileName, (short)1);
+      }
+      String bpid = cluster.getNamesystem().getBlockPoolId();
+      DataNode dn = cluster.getDataNodes().get(0);
+      Iterator<ReplicaInfo> replicasItor = 
+          dataset(dn).volumeMap.replicas(bpid).iterator();
+      ReplicaInfo replica = replicasItor.next();
+      createUnlinkTmpFile(replica, true, true); // rename block file
+      createUnlinkTmpFile(replica, false, true); // rename meta file
+      replica = replicasItor.next();
+      createUnlinkTmpFile(replica, true, false); // copy block file
+      createUnlinkTmpFile(replica, false, false); // copy meta file
+      replica = replicasItor.next();
+      createUnlinkTmpFile(replica, true, true); // rename block file
+      createUnlinkTmpFile(replica, false, false); // copy meta file
+
+      cluster.restartDataNodes();
+      cluster.waitActive();
+      dn = cluster.getDataNodes().get(0);
+
+      // check volumeMap: 4 finalized replica
+      Collection<ReplicaInfo> replicas = dataset(dn).volumeMap.replicas(bpid);
+      Assert.assertEquals(4, replicas.size());
+      replicasItor = replicas.iterator();
+      while (replicasItor.hasNext()) {
+        Assert.assertEquals(ReplicaState.FINALIZED, 
+            replicasItor.next().getState());
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   private static FsDatasetImpl dataset(DataNode dn) {
     return (FsDatasetImpl)DataNodeTestUtils.getFSDataset(dn);
   }
+
+  private static void createUnlinkTmpFile(ReplicaInfo replicaInfo, 
+      boolean changeBlockFile, 
+      boolean isRename) throws IOException {
+    File src;
+    if (changeBlockFile) {
+      src = replicaInfo.getBlockFile();
+    } else {
+      src = replicaInfo.getMetaFile();
+    }
+    File dst = DatanodeUtil.getUnlinkTmpFile(src);
+    if (isRename) {
+      src.renameTo(dst);
+    } else {
+      FileInputStream in = new FileInputStream(src);
+      try {
+        FileOutputStream out = new FileOutputStream(dst);
+        try {
+          IOUtils.copyBytes(in, out, 1);
+        } finally {
+          out.close();
+        }
+      } finally {
+        in.close();
+      }
+    }
+  }
 }


[03/37] hadoop git commit: YARN-4392. ApplicationCreatedEvent event time resets after RM restart/failover. Contributed by Naganarasimha G R and Xuan Gong

Posted by as...@apache.org.
YARN-4392. ApplicationCreatedEvent event time resets after RM
restart/failover. Contributed by Naganarasimha G R and Xuan Gong


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4546c758
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4546c758
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4546c758

Branch: refs/heads/yarn-2877
Commit: 4546c7582b6762c18ba150d80a8976eb51a8290c
Parents: 01a641b
Author: Xuan <xg...@apache.org>
Authored: Mon Dec 7 12:24:55 2015 -0800
Committer: Xuan <xg...@apache.org>
Committed: Mon Dec 7 12:24:55 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../server/resourcemanager/RMAppManager.java    |  1 -
 .../server/resourcemanager/rmapp/RMAppImpl.java | 14 ++++++++---
 .../server/resourcemanager/TestRMRestart.java   | 25 +++++++++++++++++---
 .../rmapp/TestRMAppTransitions.java             |  4 +---
 5 files changed, 37 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4546c758/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7c47126..360150a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1103,6 +1103,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4408. Fix issue that NodeManager reports negative running containers. 
     (Robert Kanter via junping_du)
 
+    YARN-4392. ApplicationCreatedEvent event time resets after RM restart/failover.
+    (Naganarasimha G R and Xuan Gong via xgong)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4546c758/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index c9ea1b8..711a7a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.ipc.CallerContext;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4546c758/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 9220849..8baddee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -444,9 +444,6 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     this.callerContext = CallerContext.getCurrent();
 
-    rmContext.getRMApplicationHistoryWriter().applicationStarted(this);
-    rmContext.getSystemMetricsPublisher().appCreated(this, startTime);
-
     long localLogAggregationStatusTimeout =
         conf.getLong(YarnConfiguration.LOG_AGGREGATION_STATUS_TIME_OUT_MS,
           YarnConfiguration.DEFAULT_LOG_AGGREGATION_STATUS_TIME_OUT_MS);
@@ -813,6 +810,9 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.startTime = appState.getStartTime();
     this.callerContext = appState.getCallerContext();
 
+    // send the ATS create Event
+    sendATSCreateEvent(this, this.startTime);
+
     for(int i=0; i<appState.getAttemptCount(); ++i) {
       // create attempt
       createNewAttempt();
@@ -1084,6 +1084,9 @@ public class RMAppImpl implements RMApp, Recoverable {
       // communication
       LOG.info("Storing application with id " + app.applicationId);
       app.rmContext.getStateStore().storeNewApplication(app);
+
+      // send the ATS create Event
+      app.sendATSCreateEvent(app, app.startTime);
     }
   }
 
@@ -1734,4 +1737,9 @@ public class RMAppImpl implements RMApp, Recoverable {
   public CallerContext getCallerContext() {
     return callerContext;
   }
+
+  private void sendATSCreateEvent(RMApp app, long startTime) {
+    rmContext.getRMApplicationHistoryWriter().applicationStarted(app);
+    rmContext.getSystemMetricsPublisher().appCreated(app, startTime);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4546c758/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index cd84208..d2b8eee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import com.google.common.base.Supplier;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -87,6 +87,7 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -113,7 +114,9 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
@@ -896,7 +899,13 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     memStore.init(conf);
 
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf, memStore) {
+      @Override
+      protected SystemMetricsPublisher createSystemMetricsPublisher() {
+        return spy(super.createSystemMetricsPublisher());
+      }
+    };
+    rms.add(rm1);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -925,6 +934,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     rm1.waitForState(app2.getApplicationId(), RMAppState.KILLED);
     rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.KILLED);
 
+    verify(rm1.getRMContext().getSystemMetricsPublisher(),Mockito.times(3))
+    .appCreated(any(RMApp.class), anyLong());
     // restart rm
 
     MockRM rm2 = new MockRM(conf, memStore) {
@@ -932,10 +943,18 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       protected RMAppManager createRMAppManager() {
         return spy(super.createRMAppManager());
       }
+
+      @Override
+      protected SystemMetricsPublisher createSystemMetricsPublisher() {
+        return spy(super.createSystemMetricsPublisher());
+      }
     };
     rms.add(rm2);
     rm2.start();
 
+    verify(rm2.getRMContext().getSystemMetricsPublisher(),Mockito.times(3))
+        .appCreated(any(RMApp.class), anyLong());
+
     GetApplicationsRequest request1 =
         GetApplicationsRequest.newInstance(EnumSet.of(
           YarnApplicationState.FINISHED, YarnApplicationState.KILLED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4546c758/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 280294f..d9f452a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.doReturn;
@@ -64,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
@@ -368,8 +368,6 @@ public class TestRMAppTransitions {
   protected RMApp testCreateAppNewSaving(
       ApplicationSubmissionContext submissionContext) throws IOException {
   RMApp application = createNewTestApp(submissionContext);
-    verify(writer).applicationStarted(any(RMApp.class));
-    verify(publisher).appCreated(any(RMApp.class), anyLong());
     // NEW => NEW_SAVING event RMAppEventType.START
     RMAppEvent event = 
         new RMAppEvent(application.getApplicationId(), RMAppEventType.START);


[29/37] hadoop git commit: HDFS-9445. Datanode may deadlock while handling a bad volume. Contributed by Walter Su.

Posted by as...@apache.org.
HDFS-9445. Datanode may deadlock while handling a bad volume. Contributed by Walter Su.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a4830179
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a4830179
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a4830179

Branch: refs/heads/yarn-2877
Commit: a48301791e9564363bc2abad4e89e344b0d7a5ff
Parents: e363417
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Dec 11 08:44:47 2015 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Dec 11 08:44:47 2015 -0600

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 89 ++++++++++++--------
 .../fsdataset/impl/TestFsDatasetImpl.java       |  4 +
 3 files changed, 59 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4830179/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1696053..c1a323b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2574,6 +2574,9 @@ Release 2.7.2 - UNRELEASED
     HDFS-9294. DFSClient deadlock when close file and failed to renew lease.
     (Brahma Reddy Battula via szetszwo)
 
+    HDFS-9445. Datanode may deadlock while handling a bad volume.
+    (Wlater Su via Kihwal)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4830179/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 1d8c705..afa4dee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -470,48 +470,67 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * Removes a set of volumes from FsDataset.
    * @param volumesToRemove a set of absolute root path of each volume.
    * @param clearFailure set true to clear failure information.
-   *
-   * DataNode should call this function before calling
-   * {@link DataStorage#removeVolumes(java.util.Collection)}.
    */
   @Override
-  public synchronized void removeVolumes(
-      Set<File> volumesToRemove, boolean clearFailure) {
+  public void removeVolumes(Set<File> volumesToRemove, boolean clearFailure) {
     // Make sure that all volumes are absolute path.
     for (File vol : volumesToRemove) {
       Preconditions.checkArgument(vol.isAbsolute(),
           String.format("%s is not absolute path.", vol.getPath()));
     }
-    for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
-      Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
-      final File absRoot = sd.getRoot().getAbsoluteFile();
-      if (volumesToRemove.contains(absRoot)) {
-        LOG.info("Removing " + absRoot + " from FsDataset.");
-
-        // Disable the volume from the service.
-        asyncDiskService.removeVolume(sd.getCurrentDir());
-        volumes.removeVolume(absRoot, clearFailure);
-
-        // Removed all replica information for the blocks on the volume. Unlike
-        // updating the volumeMap in addVolume(), this operation does not scan
-        // disks.
-        for (String bpid : volumeMap.getBlockPoolList()) {
-          for (Iterator<ReplicaInfo> it = volumeMap.replicas(bpid).iterator();
-               it.hasNext(); ) {
-            ReplicaInfo block = it.next();
-            final File absBasePath =
-                new File(block.getVolume().getBasePath()).getAbsoluteFile();
-            if (absBasePath.equals(absRoot)) {
-              invalidate(bpid, block);
-              it.remove();
+
+    Map<String, List<ReplicaInfo>> blkToInvalidate = new HashMap<>();
+    List<String> storageToRemove = new ArrayList<>();
+    synchronized (this) {
+      for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
+        Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
+        final File absRoot = sd.getRoot().getAbsoluteFile();
+        if (volumesToRemove.contains(absRoot)) {
+          LOG.info("Removing " + absRoot + " from FsDataset.");
+
+          // Disable the volume from the service.
+          asyncDiskService.removeVolume(sd.getCurrentDir());
+          volumes.removeVolume(absRoot, clearFailure);
+
+          // Removed all replica information for the blocks on the volume.
+          // Unlike updating the volumeMap in addVolume(), this operation does
+          // not scan disks.
+          for (String bpid : volumeMap.getBlockPoolList()) {
+            List<ReplicaInfo> blocks = new ArrayList<>();
+            for (Iterator<ReplicaInfo> it = volumeMap.replicas(bpid).iterator();
+                 it.hasNext(); ) {
+              ReplicaInfo block = it.next();
+              final File absBasePath =
+                  new File(block.getVolume().getBasePath()).getAbsoluteFile();
+              if (absBasePath.equals(absRoot)) {
+                blocks.add(block);
+                it.remove();
+              }
             }
+            blkToInvalidate.put(bpid, blocks);
           }
+
+          storageToRemove.add(sd.getStorageUuid());
         }
+      }
+      setupAsyncLazyPersistThreads();
+    }
 
-        storageMap.remove(sd.getStorageUuid());
+    // Call this outside the lock.
+    for (Map.Entry<String, List<ReplicaInfo>> entry :
+        blkToInvalidate.entrySet()) {
+      String bpid = entry.getKey();
+      List<ReplicaInfo> blocks = entry.getValue();
+      for (ReplicaInfo block : blocks) {
+        invalidate(bpid, block);
+      }
+    }
+
+    synchronized (this) {
+      for(String storageUuid : storageToRemove) {
+        storageMap.remove(storageUuid);
       }
     }
-    setupAsyncLazyPersistThreads();
   }
 
   private StorageType getStorageTypeFromLocations(
@@ -1931,15 +1950,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   public void invalidate(String bpid, ReplicaInfo block) {
     // If a DFSClient has the replica in its cache of short-circuit file
     // descriptors (and the client is using ShortCircuitShm), invalidate it.
-    // The short-circuit registry is null in the unit tests, because the
-    // datanode is mock object.
-    if (datanode.getShortCircuitRegistry() != null) {
-      datanode.getShortCircuitRegistry().processBlockInvalidation(
-          new ExtendedBlockId(block.getBlockId(), bpid));
+    datanode.getShortCircuitRegistry().processBlockInvalidation(
+        new ExtendedBlockId(block.getBlockId(), bpid));
 
-      // If the block is cached, start uncaching it.
-      cacheManager.uncacheBlock(bpid, block.getBlockId());
-    }
+    // If the block is cached, start uncaching it.
+    cacheManager.uncacheBlock(bpid, block.getBlockId());
 
     datanode.notifyNamenodeDeletedBlock(new ExtendedBlock(bpid, block),
         block.getStorageUuid());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4830179/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index 62907ec..a3d5769 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@@ -147,6 +148,9 @@ public class TestFsDatasetImpl {
     when(datanode.getDnConf()).thenReturn(dnConf);
     final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
     when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
+    final ShortCircuitRegistry shortCircuitRegistry =
+        new ShortCircuitRegistry(conf);
+    when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry);
 
     createStorageDirs(storage, conf, NUM_INIT_VOLUMES);
     dataset = new FsDatasetImpl(datanode, storage, conf);


[08/37] hadoop git commit: MAPREDUCE-6566. Add retry support to mapreduce CLI tool. Contributed by Varun Vasudev

Posted by as...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc470840/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index c4d2e48..37e20d3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -1,308 +1,5 @@
 Hadoop MapReduce Change Log
 
-Trunk (Unreleased)
-
-  INCOMPATIBLE CHANGES
-
-    MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically.
-    (Gera Shegalov and Karthik Kambatla via gera)
-
-    MAPREDUCE-5653. DistCp does not honour config-overrides for
-    mapreduce.[map,reduce].memory.mb (Ratandeep Ratti via aw)
-
-    MAPREDUCE-4424. 'mapred job -list' command should show the job name
-    as well. (Avinash Kujur via aajisaka)
-
-    MAPREDUCE-6336. Enable v2 FileOutputCommitter by default.
-    (Siqi Li via gera)
-
-  NEW FEATURES
-
-    MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)
-
-    MAPREDUCE-2669. Add new examples for Mean, Median, and Standard Deviation.
-    (Plamen Jeliazkov via shv)
-
-    MAPREDUCE-4887. Add RehashPartitioner, to smooth distributions
-    with poor implementations of Object#hashCode().  (Radim Kolar via cutting)
-
-    MAPREDUCE-5232. Add a configuration to be able to log classpath and other
-    system properties on mapreduce JVMs startup.  (Sangjin Lee via vinodkv)
-
-    MAPREDUCE-5910. Make MR AM resync with RM in case of work-preserving
-    RM-restart. (Rohith via jianhe)
-
-    MAPREDUCE-2841. Add a native implementation of MapOutputCollector.
-    (see section below for detailed breakdown)
-
-  IMPROVEMENTS
-
-    MAPREDUCE-3481. [Gridmix] Improve Gridmix STRESS mode. (amarrk)
-
-    MAPREDUCE-3597. [Rumen] Rumen should provide APIs to access all the
-                    job-history related information.
-
-    MAPREDUCE-3375. [Gridmix] Memory Emulation system tests.
-                    (Vinay Thota via amarrk)
-
-    MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests.
-                    (Vinay Thota via amarrk)
-
-    MAPREDUCE-2836. Provide option to fail jobs when submitted to non-existent
-    fair scheduler pools. (Ahmed Radwan via todd)
-
-    MAPREDUCE-3171. normalize nodemanager native code compilation with common/hdfs
-    native. (tucu)
-
-    MAPREDUCE-3149. Add a test to verify that TokenCache handles file system
-    uri with no authority. (John George via jitendra)
-
-    MAPREDUCE-3169. Create a new MiniMRCluster equivalent which only provides
-    client APIs cross MR1 and MR2 (Ahmed via tucu)
-
-    MAPREDUCE-2944. Improve checking of input for JobClient.displayTasks()
-    (XieXianshan via harsh)
-
-    MAPREDUCE-3956. Remove the use of the deprecated Syncable.sync() method from
-    TeraOutputFormat in the terasort example.  (szetszwo)
-
-    MAPREDUCE-3935. Annotate Counters.Counter and Counters.Group as @Public.
-    (tomwhite)
-
-    HADOOP-8285 MR changes for Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
-
-    MAPREDUCE-3302. Remove the last dependency call from
-    org.apache.hadoop.record package in MR. (harsh)
-
-    MAPREDUCE-2384. The job submitter should make sure to validate
-    jobs before creation of necessary files. (harsh)
-
-    MAPREDUCE-4371. Check for cyclic dependencies in Jobcontrol job DAG
-    (madhukara phatak via bobby)
-
-    MAPREDUCE-4686. hadoop-mapreduce-client-core fails compilation in Eclipse 
-    due to missing Avro-generated classes (Chris Nauroth via harsh)
-
-    MAPREDUCE-4735. Make arguments in TestDFSIO case insensitive.
-    (Brandon Li via suresh)
-
-    MAPREDUCE-5197. Add a service for checkpointing task state.
-    (Carlo Curino via cdouglas)
-
-    MAPREDUCE-5189. Add policies and wiring to respond to preemption requests
-    from YARN. (Carlo Curino via cdouglas)
-
-    MAPREDUCE-5196. Add bookkeeping for managing checkpoints of task state.
-    (Carlo Curino via cdouglas)
-
-    MAPREDUCE-5912. Task.calculateOutputSize does not handle Windows files after
-    MAPREDUCE-5196. (Remus Rusanu via cnauroth)
-
-    MAPREDUCE-6019. MapReduce changes for exposing YARN/MR endpoints on multiple
-    interfaces. (Craig Welch, Milan Potocnik, Arpit Agarwal via xgong)
-
-    MAPREDUCE-6013. [post-HADOOP-9902] mapred version is missing (Akira AJISAKA
-    via aw)
-
-    MAPREDUCE-6250. deprecate sbin/mr-jobhistory-daemon.sh (aw)
-
-    MAPREDUCE-6260. Convert site documentation to markdown (Masatake Iwasaki
-    via aw)
-
-    MAPREDUCE-6057. Remove obsolete entries from mapred-default.xml
-    (Ray Chiang via aw)
-
-    MAPREDUCE-2632. Avoid calling the partitioner when the numReduceTasks is 1.
-    (Ravi Teja Ch N V and Sunil G via kasha)
-
-    MAPREDUCE-6407. Migrate MAPREDUCE nativetask build to new CMake framework
-    (Alan Burlison via Colin P. McCabe)
-
-    MAPREDUCE-5485. Allow repeating job commit by extending OutputCommitter API
-    (Junping Du via jianhe)
-
-  BUG FIXES
-
-    MAPREDUCE-6191. Improve clearing stale state of Java serialization
-                    testcase.  (Sam Liu via Eric Yang)
-
-    MAPREDUCE-5714. Removed forceful JVM exit in shutDownJob.  
-                     (Jinghui Wang via Eric Yang)
-
-    MAPREDUCE-3194. "mapred mradmin" command is broken in mrv2
-                     (Jason Lowe via bobby)
-
-    MAPREDUCE-3462. Fix Gridmix JUnit testcase failures.
-                    (Ravi Prakash and Ravi Gummadi via amarrk)
-
-    MAPREDUCE-3349. Log rack-name in JobHistory for unsuccessful tasks.
-                    (Devaraj K and Amar Kamat via amarrk)
-
-    MAPREDUCE-3412. Fix 'ant docs'. (amarrk)
-
-    MAPREDUCE-3346. [Rumen] LoggedTaskAttempt#getHostName() returns null.
-                    (amarrk)
-
-    MAPREDUCE-2950. [Gridmix] TestUserResolve fails in trunk.
-                    (Ravi Gummadi via amarrk)
-
-    MAPREDUCE-2784. [Gridmix] Bug fixes in ExecutionSummarizer and
-    ResourceUsageMatcher. (amarrk)
-
-    MAPREDUCE-2978. Fixed test-patch to make Jenkins report correct number of
-    findBugs, correct links to findBugs artifacts and no links to the
-    artifacts when there are no warnings. (Tom White via vinodkv).
-
-    MAPREDUCE-3664. Federation Documentation has incorrect configuration example.
-    (Brandon Li via jitendra)
-
-    MAPREDUCE-1740. NPE in getMatchingLevelForNodes when node locations are 
-    variable depth (ahmed via tucu) [IMPORTANT: this is dead code in trunk]
-
-    MAPREDUCE-3990. MRBench allows Long-sized input-lines value
-    but parses CLI argument as an Integer. (harsh)
-
-    MAPREDUCE-3868. Make Raid Compile. (Weiyan Wang via schen)
-
-    MAPREDUCE-4685. DBCount should not use ACCESS. (Viji via harsh)
-
-    MAPREDUCE-3223. Remove MR1 configs from mapred-default.xml (tlipcon via harsh)
-
-    MAPREDUCE-4695. Fix LocalRunner on trunk after MAPREDUCE-3223 broke it
-    (harsh)
-
-    MAPREDUCE-3914. Mismatched free() / delete / delete [] in HadoopPipes
-    (Joe Mudd via aw)
-
-    MAPREDUCE-1125. SerialUtils.cc: deserializeFloat is out of sync with
-    SerialUtils.hh (Simone Leo via aw)
-
-    MAPREDUCE-4574. Fix TotalOrderParitioner to work with
-    non-WritableComparable key types. (harsh)
-
-    MAPREDUCE-5012. Typo in javadoc for IdentityMapper class. (Adam Monsen
-    via suresh)
-
-    MAPREDUCE-4987. TestMRJobs#testDistributedCache fails on Windows due to
-    classpath problems and unexpected behavior of symlinks (Chris Nauroth via
-    bikas)
-
-    MAPREDUCE-5191. TestQueue#testQueue fails with timeout on Windows. (Ivan
-    Mitic via hitesh)
-
-    MAPREDUCE-5717. Task pings are interpreted as task progress (jlowe)
-
-    MAPREDUCE-5867. Fix NPE in KillAMPreemptionPolicy related to 
-    ProportionalCapacityPreemptionPolicy (Sunil G via devaraj)
-
-    MAPREDUCE-5972. Fix typo 'programatically' in job.xml (and a few other
-    places) (Akira AJISAKA via aw)
-
-    MAPREDUCE-6161. mapred hsadmin command missing from trunk (Allen Wittenauer
-    via jlowe)
-
-    MAPREDUCE-4413. MR lib dir contains jdiff (which is gpl) (Nemon Lou via aw)
-
-    MAPREDUCE-6234. TestHighRamJob fails due to the change in MAPREDUCE-5785. 
-    (Masatake Iwasaki via kasha)
-
-    MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value 
-    greater than 2GB expressed in bytes. (Hao Xia via kasha)
-
-    MAPREDUCE-6396. TestPipeApplication fails by NullPointerException.
-    (Brahma Reddy Battula via aajisaka)
-
-    MAPREDUCE-6406. Update FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT
-    to match mapred-default.xml. (Ray Chiang via devaraj)
-
-    MAPREDUCE-6257. Document encrypted spills (Bibin A Chundatt via aw)
-
-    MAPREDUCE-6391. util/Timer.cc completely misunderstands _POSIX_CPUTIME
-    (Alan Burlison via aw)
-
-    MAPREDUCE-6412. Make hadoop-mapreduce-client Native code -Wall-clean
-    (Alan Burlison via aw)
-
-    MAPREDUCE-6416. Not all platforms have d_type in struct dirent
-    (Alan Burlison via aw)
-
-    MAPREDUCE-6435. MapReduce client assumes the world is x86
-    (Alan Burlison via aw)
-
-    MAPREDUCE-6540. TestMRTimelineEventHandling fails (sjlee)
-
-    MAPREDUCE-6555. TestMRAppMaster fails on trunk. (Junping Du via ozawa)
-
-  BREAKDOWN OF MAPREDUCE-2841 (NATIVE TASK) SUBTASKS
-
-    MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by
-    Binglin Chang
-
-    MAPREDUCE-5994. Simplify ByteUtils and fix failing test. (todd)
-
-    MAPREDUCE-5996. native-task: Rename system tests into standard directory
-    layout (todd)
-
-    MAPREDUCE-5997. native-task: Use DirectBufferPool from Hadoop Common (todd)
-
-    MAPREDUCE-6000. native-task: Simplify ByteBufferDataReader/Writer (todd)
-
-    MAPREDUCE-5991. native-task should not run unit tests if native profile is
-    not enabled. (Binglin Chang)
-
-    MAPREDUCE-5995. native-task: Revert changes to Text internals (todd)
-
-    MAPREDUCE-6005. native-task: Fix some valgrind errors (Binglin Chang)
-
-    MAPREDUCE-5984. native-task: Reuse lz4 sources in hadoop-common (Binglin
-    Chang)
-
-    MAPREDUCE-5976. native-task: should not fail to build if snappy is missing
-    (Manu Zhang)
-
-    MAPREDUCE-5978. native-task: remove test case for not supported codec
-    Bzip2Codec and DefaultCodec (Manu Zhang)
-
-    MAPREDUCE-6006. native-task: add native tests to maven and fix bug in
-    pom.xml (Binglin Chang via todd)
-
-    MAPREDUCE-6026. native-task: fix logging (Manu Zhang via todd)
-
-    MAPREDUCE-6035. native-task: sources/test-sources jar distribution (Manu
-    Zhang via todd)
-
-    MAPREDUCE-5977. Fix or suppress native-task gcc warnings (Manu Zhang via
-    todd)
-
-    MAPREDUCE-6054. native-task: Speed up tests (todd)
-
-    MAPREDUCE-6058. native-task: KVTest and LargeKVTest should check mr job is
-    sucessful (Binglin Chang)
-
-    MAPREDUCE-6056. native-task: move system test working dir to target dir and
-    cleanup test config xml files (Manu Zhang via bchang)
-
-    MAPREDUCE-6055. native-task: findbugs, interface annotations, and other misc
-    cleanup (todd)
-
-    MAPREDUCE-6067. native-task: fix some counter issues (Binglin Chang)
-
-    MAPREDUCE-6069. native-task: Lint/style fixes and removal of unused code
-    (todd)
-
-    MAPREDUCE-6074. native-task: fix release audit, javadoc, javac warnings
-    (todd)
-
-    MAPREDUCE-6077. native-task: Remove CustomModule examples in nativetask (seanzhong)
-
-    MAPREDUCE-6078. native-task: fix gtest build on macosx (Binglin Chang)
-
-    MAPREDUCE-5801. Uber mode's log message is missing a vcore reason
-    (Steven Wong via aw)
-
-    MAPREDUCE-6525. Fix test failure of TestMiniMRClientCluster.testRestart.
-    (Masatake Iwasaki via aajisaka)
-
 Release 2.9.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -424,6 +121,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6384. Add the last reporting reducer info for too many fetch
     failure diagnostics (Chang Li via jlowe)
 
+    MAPREDUCE-5762. Port MAPREDUCE-3223 and MAPREDUCE-4695 (Remove MRv1 config
+    from mapred-default.xml) to branch-2. (aajisaka)
+
     MAPREDUCE-6443. Add JvmPauseMonitor to JobHistoryServer. (Robert Kanter
     via junping_du)
 
@@ -445,6 +145,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5763. Warn message about httpshuffle in NM logs.
     (Akira AJISAKA via ozawa)
 
+    MAPREDUCE-5485. Allow repeating job commit by extending OutputCommitter
+    API. Contributed by Junping Du
+
     MAPREDUCE-6499. Add elapsed time for retired job in JobHistoryServer WebUI.
     (Lin Yiqun via aajisaka)
 
@@ -563,6 +266,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6382. Don't escape HTML links in Diagnostics in JHS job overview.
     (Siqi Li via gera)
 
+    MAPREDUCE-5232. Add a configuration to be able to log classpath and other
+    system properties on mapreduce JVMs startup.  (Sangjin Lee via vinodkv)
+
     MAPREDUCE-6388. Remove deprecation warnings from JobHistoryServer classes
     (Ray Chiang via ozawa).
 
@@ -656,6 +362,8 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6533. testDetermineCacheVisibilities of
     TestClientDistributedCacheManager is broken (Chang Li via jlowe)
 
+    MAPREDUCE-6540. TestMRTimelineEventHandling fails (sjlee)
+
     MAPREDUCE-6553. Replace '\u2b05' with '<-' in rendering job configuration.
     (Gabor Liptak via aajisaka)
 
@@ -672,6 +380,7 @@ Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
+
   NEW FEATURES
 
   IMPROVEMENTS
@@ -747,6 +456,8 @@ Release 2.7.2 - UNRELEASED
     MAPREDUCE-6451. DistCp has incorrect chunkFilePath for multiple jobs when
     strategy is dynamic (Kuhu Shukla via kihwal)
 
+    MAPREDUCE-6566. Add retry support to mapreduce CLI tool. (Varun Vasudev via xgong)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES
@@ -941,17 +652,14 @@ Release 2.7.0 - 2015-04-20
     MAPREDUCE-2815. JavaDoc does not generate correctly for
     MultithreadedMapRunner. (Chris Palmer via aajisaka)
 
-    MAPREDUCE-6223. TestJobConf#testNegativeValueForTaskVmem failures. 
-    (Varun Saxena via kasha)
-
     MAPREDUCE-6268. Fix typo in Task Attempt API's URL. (Ryu Kobayashi
     via ozawa)
 
     MAPREDUCE-6136. MRAppMaster doesn't shutdown file systems. (Brahma 
     Reddy Battula via ozawa)
 
-    MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc
-    comments. (Akira AJISAKA and Andrew Purtell via ozawa)
+    MAPREDUCE-5657. [JDK8] Fix Javadoc errors caused by incorrect or illegal
+    tags in doc comments. (Akira AJISAKA and Andrew Purtell via ozawa)
 
     MAPREDUCE-4742. Fix typo in nnbench#displayUsage. (Liang Xie via ozawa)
 
@@ -1081,6 +789,8 @@ Release 2.6.0 - 2014-11-18
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
+    MAPREDUCE-5910. Make MR AM resync with RM in case of work-preserving
+    RM-restart. Contributed by Rohith
 
     MAPREDUCE-5933. Enabled MR AM to post history events to the timeline server.
     (Robert Kanter via zjshen)
@@ -1093,13 +803,16 @@ Release 2.6.0 - 2014-11-18
     MAPREDUCE-5963. ShuffleHandler DB schema should be versioned with
     compatible/incompatible changes (Junping Du via jlowe)
 
+    MAPREDUCE-6019. MapReduce changes for exposing YARN/MR endpoints on multiple
+    interfaces. (Craig Welch, Milan Potocnik, Arpit Agarwal via xgong)
+
     MAPREDUCE-883. harchive: Document how to unarchive (Akira AJISAKA and
       Koji Noguchi via aw)
 
-    MAPREDUCE-4791. Javadoc for KeyValueTextInputFormat should include default
+    MAPREDUCE-4791. Javadoc for KeyValueTextInputFormat should include default 
       separator and how to change it (Akira AJISAKA via aw)
 
-    MAPREDUCE-5906. Inconsistent configuration in property
+    MAPREDUCE-5906. Inconsistent configuration in property 
       "mapreduce.reduce.shuffle.input.buffer.percent" (Akira AJISAKA via aw)
 
     MAPREDUCE-5974. Allow specifying multiple MapOutputCollectors with 
@@ -1148,27 +861,24 @@ Release 2.6.0 - 2014-11-18
     MAPREDUCE-6021. MR AM should have working directory in LD_LIBRARY_PATH
     (jlowe)
 
-    MAPREDUCE-6010. HistoryServerFileSystemStateStore fails to update tokens
-    (jlowe)
-
-    MAPREDUCE-5878. some standard JDK APIs are not part of system classes
-    defaults (Sangjin Lee via jlowe)
-
     MAPREDUCE-5944. Remove MRv1 commands from CommandsManual.apt.vm
       (Akira AJISAKA via aw)
 
     MAPREDUCE-5943. Separate mapred commands from CommandManual.apt.vm
       (Akira AJISAKA via aw)
 
-    MAPREDUCE-5363. Fix doc and spelling for TaskCompletionEvent#getTaskStatus
+    MAPREDUCE-5363. Fix doc and spelling for TaskCompletionEvent#getTaskStatus 
       and getStatus (Akira AJISAKA via aw)
 
+    MAPREDUCE-6010. HistoryServerFileSystemStateStore fails to update tokens
+    (jlowe)
+
     MAPREDUCE-5595. Typo in MergeManagerImpl.java (Akira AJISAKA via aw)
 
-    MAPREDUCE-5597. Missing alternatives in javadocs for deprecated constructors
-     in mapreduce.Job (Akira AJISAKA via aw)
+    MAPREDUCE-5597. Missing alternatives in javadocs for deprecated constructors 
+      in mapreduce.Job (Akira AJISAKA via aw)
 
-    MAPREDUCE-5950. incorrect description in distcp2 document (Akira AJISAKA
+    MAPREDUCE-5950. incorrect description in distcp2 document (Akira AJISAKA 
       via aw)
 
     MAPREDUCE-5998. CompositeInputFormat javadoc is broken (Akira AJISAKA via
@@ -1176,6 +886,9 @@ Release 2.6.0 - 2014-11-18
 
     MAPREDUCE-5999. Fix dead link in InputFormat javadoc (Akira AJISAKA via aw)
 
+    MAPREDUCE-5878. some standard JDK APIs are not part of system classes
+    defaults (Sangjin Lee via jlowe)
+
     MAPREDUCE-6032. Made MR jobs write job history files on the default FS when
     the current context's FS is different. (Benjamin Zhitomirsky via zjshen)
 
@@ -1238,7 +951,7 @@ Release 2.6.0 - 2014-11-18
     MAPREDUCE-6104. TestJobHistoryParsing.testPartialJob fails in branch-2
     (Mit Desai via jlowe)
 
-    MAPREDUCE-6109. Fix minor typo in distcp -p usage text (Charles Lamb 
+    MAPREDUCE-6109. Fix minor typo in distcp -p usage text (Charles Lamb
     via aw)
 
     MAPREDUCE-6093. minor distcp doc edits (Charles Lamb via aw)
@@ -1246,7 +959,7 @@ Release 2.6.0 - 2014-11-18
     MAPREDUCE-5831. Make MR client ignore unknown counters received from AM.
     (Junping Du via zjshen)
 
-    MAPREDUCE-6073. Description of mapreduce.job.speculative.slowtaskthreshold 
+    MAPREDUCE-6073. Description of mapreduce.job.speculative.slowtaskthreshold
     in mapred-default should be moved into description tags (Tsuyoshi OZAWA 
     via aw)
 
@@ -1416,7 +1129,7 @@ Release 2.5.0 - 2014-08-11
 
     MAPREDUCE-5759. Remove unnecessary conf load in Limits (Sandy Ryza)
 
-    MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
+    MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing. 
     (Srikanth Sundarrajan via amareshwari)
 
     MAPREDUCE-5775. Remove unnecessary job.setNumReduceTasks in SleepJob.createJob 
@@ -1499,6 +1212,9 @@ Release 2.4.1 - 2014-06-23
 
   BUG FIXES
 
+    MAPREDUCE-5714. Removed forceful JVM exit in shutDownJob.  
+                     (Jinghui Wang via Eric Yang)
+
     MAPREDUCE-5818. Added "hsadmin" command into mapred.cmd. (Jian He via zjshen)
 
     MAPREDUCE-5824. Fixed test-failure of TestPipesNonJavaInputFormat in
@@ -1560,9 +1276,6 @@ Release 2.4.0 - 2014-04-07
     MAPREDUCE-5754. Preserve Job diagnostics in history (Gera Shegalov via
     jlowe)
 
-    MAPREDUCE-5766. Moved ping messages from TaskAttempts to be at DEBUG level
-    inside the ApplicationMaster log. (Jian He via vinodkv)
-
     MAPREDUCE-5773. Provide dedicated MRAppMaster syslog length limit (Gera
     Shegalov via jlowe)
 
@@ -2677,9 +2390,6 @@ Release 2.0.3-alpha - 2013-02-06
 
     MAPREDUCE-4654. TestDistCp is ignored. (Sandy Ryza via tomwhite)
 
-    MAPREDUCE-4736. Remove obsolete option [-rootDir] from TestDFSIO.
-    (Brandon Li via suresh)
-
     MAPREDUCE-4637. Handle TaskAttempt diagnostic updates while in the NEW and 
     UNASSIGNED states. (Mayank Bansal via sseth)
 
@@ -2697,6 +2407,9 @@ Release 2.0.3-alpha - 2013-02-06
     MAPREDUCE-4856. TestJobOutputCommitter uses same directory as
     TestJobCleanup. (Sandy Ryza via tomwhite)
 
+    MAPREDUCE-4895. Fix compilation failure of org.apache.hadoop.mapred.
+    gridmix.TestResourceUsageEmulators (Dennis Y via tgraves)
+
     MAPREDUCE-4278. Cannot run two local jobs in parallel from the same
     gateway. (Sandy Ryza via tomwhite)
 
@@ -2768,23 +2481,11 @@ Release 2.0.2-alpha - 2012-09-07
     MAPREDUCE-3921. MR AM should act on node health status changes. 
     (Bikas Saha via sseth)
 
-    MAPREDUCE-2220. Fix new API FileOutputFormat-related typos in
-    mapred-default.xml (Rui Kubo via harsh)
-
-    MAPREDUCE-3907. Document entries mapred-default.xml for the
-    jobhistory server. (Eugene Koontz via harsh)
-
-    MAPREDUCE-3906. Fix inconsistency in documentation regarding
-    mapreduce.jobhistory.principal. (Eugene Koontz via harsh)
-
-    MAPREDUCE-4432. Confusing warning message when GenericOptionsParser
-    is not used. (Gabriel Reid via harsh)
+    MAPREDUCE-4355. Add RunningJob.getJobStatus() (kkambatl via tucu)
 
     MAPREDUCE-4427. Added an 'unmanaged' mode for AMs so as to ease
     development of new applications. (Bikas Saha via acmurthy) 
 
-    MAPREDUCE-4447. Remove aop from cruft from the ant build. (eli)
-
     MAPREDUCE-3289. Make use of fadvise in the NM's shuffle handler.
     (Todd Lipcon and Siddharth Seth via sseth)
 
@@ -2830,7 +2531,7 @@ Release 2.0.2-alpha - 2012-09-07
     classpath (rkanter via tucu)
 
     MAPREDUCE-4577. HDFS-3672 broke
-    TestCombineFileInputFormat.testMissingBlocks() test. (atm)
+    TestCombineFileInputFormat.testMissingBlocks() test (atm)
 
     MAPREDUCE-4470. Fix TestCombineFileInputFormat.testForEmptyFile (ikatsov via tucu)
 
@@ -2946,6 +2647,9 @@ Release 2.0.0-alpha - 05-23-2012
     MAPREDUCE-3885. Avoid an unnecessary copy for all requests/responses in 
     MRs ProtoOverHadoopRpcEngine. (Devaraj Das via sseth)
 
+	  MAPREDUCE-3935. Annotate Counters.Counter and Counters.Group as @Public.
+    (tomwhite)
+
     MAPREDUCE-3991. Streaming FAQ has some wrong instructions about input files 
     splitting. (harsh)
 
@@ -2978,6 +2682,20 @@ Release 2.0.0-alpha - 05-23-2012
     MAPREDUCE-4205. retrofit all JVM shutdown hooks to use ShutdownHookManager 
     (tucu)
 
+    HADOOP-8285 MR changes for Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
+
+    MAPREDUCE-2220. Fix new API FileOutputFormat-related typos in
+    mapred-default.xml (Rui Kubo via harsh)
+
+    MAPREDUCE-3907. Document entries mapred-default.xml for the
+    jobhistory server. (Eugene Koontz via harsh)
+
+    MAPREDUCE-3906. Fix inconsistency in documentation regarding
+    mapreduce.jobhistory.principal. (Eugene Koontz via harsh)
+
+    MAPREDUCE-4432. Confusing warning message when GenericOptionsParser
+    is not used. (Gabriel Reid via harsh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -3034,9 +2752,6 @@ Release 2.0.0-alpha - 05-23-2012
 
     MAPREDUCE-4091. tools testcases failing because of MAPREDUCE-4082 (tucu)
 
-    MAPREDUCE-4095. TestJobInProgress#testLocality uses a bogus topology.
-    (Colin Patrick McCabe via eli)
-
     MAPREDUCE-4098. TestMRApps testSetClasspath fails (tucu)
 
     MAPREDUCE-4097. tools testcases fail because missing mrapp-generated-classpath 
@@ -3231,7 +2946,7 @@ Release 0.23.7 - 2013-04-18
 
     MAPREDUCE-5027. Shuffle does not limit number of outstanding connections
     (Robert Parker via jeagles)
-    
+
     MAPREDUCE-4972. Coverage fixing for org.apache.hadoop.mapreduce.jobhistory
     (Aleksey Gorshkov via bobby)
 
@@ -3278,7 +2993,7 @@ Release 0.23.7 - 2013-04-18
 
     MAPREDUCE-4991. coverage for gridmix (Aleksey Gorshkov via tgraves)
 
-    MAPREDUCE-5007. fix coverage org.apache.hadoop.mapreduce.v2.hs (Aleksey 
+    MAPREDUCE-5007. fix coverage org.apache.hadoop.mapreduce.v2.hs (Aleksey
     Gorshkov via tgraves)
 
     MAPREDUCE-5137. AM web UI: clicking on Map Task results in 500 error
@@ -3851,6 +3566,9 @@ Release 0.23.3
     MAPREDUCE-4641. Exception in commitJob marks job as successful in job
     history (Jason Lowe via bobby)
 
+    MAPREDUCE-4549. Distributed cache conflicts breaks backwards compatability 
+    (Robert Evans via tucu)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -3862,7 +3580,7 @@ Release 0.23.2 - UNRELEASED
     MAPREDUCE-3849. Change TokenCache's reading of the binary token file
     (Daryn Sharp via bobby)
 
-    MAPREDUCE-3854. Fixed and reenabled tests related to MR child JVM's
+    MAPREDUCE-3854. Fixed and reenabled tests related to MR child JVM's           
     environmental variables in TestMiniMRChildTask. (Tom White via vinodkv)
 
     MAPREDUCE-3877 Add a test to formalise the current state transitions
@@ -3910,21 +3628,19 @@ Release 0.23.2 - UNRELEASED
     sseth)
 
   BUG FIXES
+
     MAPREDUCE-3918  proc_historyserver no longer in command line arguments for
     HistoryServer (Jon Eagles via bobby)
 
     MAPREDUCE-3862.  Nodemanager can appear to hang on shutdown due to lingering
     DeletionService threads (Jason Lowe via bobby)
 
-    MAPREDUCE-3680. FifoScheduler web service rest API can print out invalid
+    MAPREDUCE-3680. FifoScheduler web service rest API can print out invalid 
     JSON. (B Anil Kumar via tgraves)
 
-    MAPREDUCE-3852. Test TestLinuxResourceCalculatorPlugin failing. (Thomas
+    MAPREDUCE-3852. Test TestLinuxResourceCalculatorPlugin failing. (Thomas 
     Graves via mahadev)
 
-    MAPREDUCE-3736. Variable substitution depth too large for fs.default.name
-    causes jobs to fail (ahmed via tucu).
-
     MAPREDUCE-3864. Fix cluster setup docs for correct SecondaryNameNode
     HTTPS parameters. (todd)
 
@@ -3946,7 +3662,7 @@ Release 0.23.2 - UNRELEASED
     dies unexpectedly (Jason Lowe via sseth)
 
     MAPREDUCE-3904 Job history produced with mapreduce.cluster.acls.enabled
-    false can not be viewed with mapreduce.cluster.acls.enabled true
+    false can not be viewed with mapreduce.cluster.acls.enabled true 
     (Jonathon Eagles via tgraves)
 
     MAPREDUCE-3910. Fixed a bug in CapacityScheduler LeafQueue which was causing
@@ -3956,9 +3672,9 @@ Release 0.23.2 - UNRELEASED
     zero counter values for framework counters. (Bhallamudi Venkata Siva Kamesh
     via vinodkv)
 
-    MAPREDUCE-3913. RM application webpage is unresponsive after 2000 jobs
+    MAPREDUCE-3913. RM application webpage is unresponsive after 2000 jobs 
     (Jason Lowe via tgraves)
-
+ 
     MAPREDUCE-3922. Fixed build to not compile 32bit container-executor binary
     by default on all platforms. (Hitesh Shah via vinodkv)
 
@@ -4038,20 +3754,39 @@ Release 0.23.2 - UNRELEASED
 
 Release 0.23.1 - 2012-02-17
 
-  INCOMPATIBLE CHANGES
-
   NEW FEATURES
 
-   MAPREDUCE-3121. NodeManager should handle disk-failures (Ravi Gummadi via mahadev)
+    MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)
+
+    MAPREDUCE-3121. NodeManager should handle disk-failures (Ravi Gummadi via mahadev)
 
-   MAPREDUCE-2863. Support web services for YARN and MR components. (Thomas
-   Graves via vinodkv)
+    MAPREDUCE-2863. Support web services for YARN and MR components. (Thomas
+    Graves via vinodkv)
 
-   MAPREDUCE-3251. Network ACLs can prevent some clients to talk to MR ApplicationMaster
-   (Anupam Seth via mahadev)
+    MAPREDUCE-3251. Network ACLs can prevent some clients to talk to MR ApplicationMaster.
+    (Anupam Seth via mahadev)
+
+    MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)
 
   IMPROVEMENTS
 
+    MAPREDUCE-3481. [Gridmix] Improve Gridmix STRESS mode. (amarrk)
+
+    MAPREDUCE-3597. [Rumen] Rumen should provide APIs to access all the 
+                    job-history related information.
+
+    MAPREDUCE-3375. [Gridmix] Memory Emulation system tests. 
+                    (Vinay Thota via amarrk)
+
+    MAPREDUCE-3840.  JobEndNotifier doesn't use the proxyToUse during connecting
+    (Ravi Prakash via bobby)
+
+    MAPREDUCE-3736. Variable substitution depth too large for fs.default.name 
+    causes jobs to fail (ahmed via tucu).
+ 
+    MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests. 
+                    (Vinay Thota via amarrk)
+
     MAPREDUCE-3297. Moved log related components into yarn-common so that
     HistoryServer and clients can use them without depending on the
     yarn-server-nodemanager module. (Siddharth Seth via vinodkv)
@@ -4065,23 +3800,26 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3370. Fixed MiniMRYarnCluster and related tests to not use
     a hard-coded path for the mr-app jar. (Ahmed Radwan via vinodkv)
 
-    MAPREDUCE-3325. Improvements to CapacityScheduler doc. (Thomas Graves
+    MAPREDUCE-3325. Improvements to CapacityScheduler doc. (Thomas Graves 
     via mahadev)
 
     MAPREDUCE-3341. Enhance logging of initalized queue limit values.
     (Anupam Seth via mahadev)
 
-    MAPREDUCE-3243. Invalid tracking URL for streaming jobs (Jonathan Eagles
+    MAPREDUCE-3243. Invalid tracking URL for streaming jobs (Jonathan Eagles 
     via mahadev)
 
-    MAPREDUCE-3331. Improvement to single node cluster setup documentation for
+    MAPREDUCE-3331. Improvement to single node cluster setup documentation for 
     0.23 (Anupam Seth via mahadev)
 
     MAPREDUCE-3102. Changed NodeManager to fail fast when LinuxContainerExecutor
     has wrong configuration or permissions. (Hitesh Shah via vinodkv)
 
-    MAPREDUCE-3415. improve MiniMRYarnCluster & DistributedShell JAR resolution.
-    (tucu)
+    MAPREDUCE-3415. improve MiniMRYarnCluster & DistributedShell JAR
+    resolution. (tucu)
+
+    MAPREDUCE-3169. Create a new MiniMRCluster equivalent which only provides 
+    client APIs cross MR1 and MR2. (Ahmed via tucu)
 
     MAPREDUCE-3373. Hadoop scripts unconditionally source
     "$bin"/../libexec/hadoop-config.sh. (Bruno Mahé via tomwhite)
@@ -4089,7 +3827,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3372. HADOOP_PREFIX cannot be overridden.
     (Bruno Mahé via tomwhite)
 
-    MAPREDUCE-3411. Performance Upgrade for jQuery (Jonathan Eagles via
+    MAPREDUCE-3411. Performance Upgrade for jQuery (Jonathan Eagles via 
     mahadev)
 
     MAPREDUCE-3371. Review and improve the yarn-api javadocs. (Ravi Prakash
@@ -4112,7 +3850,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3369. Migrate MR1 tests to run on MR2 using the new interfaces
     introduced in MAPREDUCE-3169. (Ahmed Radwan via tomwhite)
 
-    MAPREDUCE-3518. mapred queue -info <queue> -showJobs throws NPE.
+    MAPREDUCE-3518. mapred queue -info <queue> -showJobs throws NPE. 
     (Jonathan Eagles via mahadev)
 
     MAPREDUCE-3391. Making a trivial change to correct a log message in
@@ -4143,10 +3881,10 @@ Release 0.23.1 - 2012-02-17
     Improved the earlier patch to not to JobHistoryServer repeatedly.
     (Anupam Seth via vinodkv)
 
-    MAPREDUCE-3553. Add support for data returned when exceptions thrown from web
+    MAPREDUCE-3553. Add support for data returned when exceptions thrown from web 
     service apis to be in either xml or in JSON. (Thomas Graves via mahadev)
 
-    MAPREDUCE-3641. Making CapacityScheduler more conservative so as to
+    MAPREDUCE-3641. Making CapacityScheduler more conservative so as to 
     assign only one off-switch container in a single scheduling
     iteration. (Arun C Murthy via vinodkv)
 
@@ -4164,12 +3902,12 @@ Release 0.23.1 - 2012-02-17
     values for working at scale. (Hitesh Shah via vinodkv)
 
     MAPREDUCE-3693. Added mapreduce.admin.user.env to mapred-default.xml.
-    (Roman Shapshonik via acmurthy)
+    (Roman Shapshonik via acmurthy) 
 
     MAPREDUCE-3732. Modified CapacityScheduler to use only users with pending
     requests for computing user-limits. (Arun C Murthy via vinodkv)
 
-    MAPREDUCE-3679. AM logs and others should not automatically refresh after every 1
+    MAPREDUCE-3679. AM logs and others should not automatically refresh after every 1 
     second. (Vinod KV  via mahadev)
 
     MAPREDUCE-3754. Modified RM UI to filter applications based on state of the
@@ -4215,13 +3953,29 @@ Release 0.23.1 - 2012-02-17
     on the webapps. (Bhallamudi Venkata Siva Kamesh and Jason Lowe via vinodkv)
 
     MAPREDUCE-3756. Made single shuffle limit configurable. (Hitesh Shah via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-3811. Made jobclient-to-AM retries configurable. (sseth via
-    acmurthy)
+    acmurthy) 
 
   BUG FIXES
 
+    MAPREDUCE-2784. [Gridmix] Bug fixes in ExecutionSummarizer and 
+                    ResourceUsageMatcher. (amarrk)
+
+    MAPREDUCE-3194. "mapred mradmin" command is broken in mrv2
+                     (Jason Lowe via bobby)
+
+    MAPREDUCE-3462. Fix Gridmix JUnit testcase failures. 
+                    (Ravi Prakash and Ravi Gummadi via amarrk)
+
+    MAPREDUCE-2950. [Rumen] Fixed TestUserResolve. (Ravi Gummadi via amarrk)
+
+    MAPREDUCE-3412. Fix 'ant docs'. (amarrk)
+
+    MAPREDUCE-3346 [Rumen] LoggedTaskAttempt#getHostName() returns null.
+                   (amarrk)
+
     MAPREDUCE-3221. Reenabled the previously ignored test in TestSubmitJob
     and fixed bugs in it. (Devaraj K via vinodkv)
 
@@ -4234,7 +3988,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3217. Reenabled and fixed bugs in the failing ant test
     TestAuditLogger. (Devaraj K via vinodkv)
 
-    MAPREDUCE-3291. App fail to launch due to delegation token not
+    MAPREDUCE-3291. App fail to launch due to delegation token not 
     found in cache (Robert Evans via mahadev)
 
     MAPREDUCE-3344. o.a.h.mapreduce.Reducer since 0.21 blindly casts to
@@ -4246,6 +4000,8 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3345. Fixed a race condition in ResourceManager that was causing
     TestContainerManagerSecurity to fail sometimes. (Hitesh Shah via vinodkv)
 
+    MAPREDUCE-3368. Fixed test compilation. (Hitesh Shah via vinodkv)
+
     MAPREDUCE-3333. Fixed bugs in ContainerLauncher of MR AppMaster due to
     which per-container connections to NodeManager were lingering long enough
     to hit the ulimits on number of processes. (vinodkv)
@@ -4256,7 +4012,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3379. Fixed LocalResourceTracker in NodeManager to remove deleted
     cache entries correctly. (Siddharth Seth via vinodkv)
 
-    MAPREDUCE-3324. Not All HttpServer tools links (stacks,logs,config,metrics) are
+    MAPREDUCE-3324. Not All HttpServer tools links (stacks,logs,config,metrics) are 
     accessible through all UI servers (Jonathan Eagles via mahadev)
 
     MAPREDUCE-3355. Fixed MR AM's ContainerLauncher to handle node-command
@@ -4265,6 +4021,9 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3407. Fixed pom files to refer to the correct MR app-jar needed
     by the integration tests. (Hitesh Shah via vinodkv)
 
+    MAPREDUCE-3437. Fix examples pom to refer to the correct 0.23 snapshot
+    version. (Jonathan Eagles via todd)
+
     MAPREDUCE-3434. Nightly build broken (Hitesh Shah via mahadev)
 
     MAPREDUCE-3447. mapreduce examples not working (mahadev)
@@ -4274,7 +4033,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3454. [Gridmix] TestDistCacheEmulation is broken (Hitesh Shah
     via mahadev)
 
-    MAPREDUCE-3408. yarn-daemon.sh unconditionnaly sets yarn.root.logger
+    MAPREDUCE-3408. yarn-daemon.sh unconditionnaly sets yarn.root.logger 
     (Bruno Mahe via mahadev)
 
     MAPREDUCE-3329. Fixed CapacityScheduler to ensure maximum-capacity cannot
@@ -4283,7 +4042,10 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3464. mapreduce jsp pages missing DOCTYPE. (Dave Vronay via mattf)
 
     MAPREDUCE-3265. Removed debug logs during job submission to LOG.debug to
-    cut down noise. (acmurthy)
+    cut down noise. (acmurthy) 
+
+    MAPREDUCE-3468. Changed ant based infrastructure to use 0.23.1 version.
+    (sseth via acmurthy) 
 
     MAPREDUCE-3433. Finding counters by legacy group name returns empty
     counters. (tomwhite)
@@ -4291,9 +4053,12 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3450. NM port info no longer available in JobHistory.
     (Siddharth Seth via mahadev)
 
+    MAPREDUCE-3477. Hadoop site documentation cannot be built anymore. 
+    (jeagles via tucu)
+
     MAPREDUCE-3488. Streaming jobs are failing because the main class
     isnt set in the pom files. (mahadev)
-
+ 
     MAPREDUCE-3463. Second AM fails to recover properly when first AM is killed with
     java.lang.IllegalArgumentException causing lost job. (Siddharth Seth via mahadev)
 
@@ -4311,22 +4076,27 @@ Release 0.23.1 - 2012-02-17
 
     MAPREDUCE-3479. JobClient#getJob cannot find local jobs. (tomwhite)
 
-    MAPREDUCE-3456. $HADOOP_PREFIX/bin/yarn should set defaults for
+    MAPREDUCE-3500. MRJobConfig creates an LD_LIBRARY_PATH using the platform ARCH. (tucu)
+
+    MAPREDUCE-3456. $HADOOP_PREFIX/bin/yarn should set defaults for 
     $HADOOP_*_HOME (Eric Payne via mahadev)
 
     MAPREDUCE-3458. Fix findbugs warnings in hadoop-examples. (Devaraj K
     via mahadev)
 
-    MAPREDUCE-3485. DISKS_FAILED -101 error code should be defined in same location as
+    MAPREDUCE-3485. DISKS_FAILED -101 error code should be defined in same location as 
     ABORTED_CONTAINER_EXIT_STATUS. (Ravi Gummadi via mahadev)
 
+    MAPREDUCE-3389. MRApps loads the 'mrapp-generated-classpath' file with 
+    classpath from the build machine. (tucu)
+
     MAPREDUCE-3496. Fixed client to print queue acls in consistent order.
-    (Jonathan Eagles via acmurthy)
+    (Jonathan Eagles via acmurthy) 
 
-    MAPREDUCE-3147. Handle leaf queues with the same name properly. (Ravi Prakash via
-    mahadev)
+    MAPREDUCE-3147. Handle leaf queues with the same name properly.
+    (Ravi Prakash via mahadev)
 
-    MAPREDUCE-3327. RM web ui scheduler link doesn't show correct max value
+    MAPREDUCE-3327. RM web ui scheduler link doesn't show correct max value 
     for queues (Anupam Seth via mahadev)
 
     MAPREDUCE-3513. Capacity Scheduler web UI has a spelling mistake for Memory.
@@ -4338,10 +4108,10 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3527. Fix minor API incompatibilities between 1.0 and 0.23.
     (tomwhite)
 
-    MAPREDUCE-3328. mapred queue -list output inconsistent and missing child
+    MAPREDUCE-3328. mapred queue -list output inconsistent and missing child 
     queues. (Ravi Prakash via mahadev)
 
-    MAPREDUCE-3510. Capacity Scheduler inherited ACLs not displayed by mapred queue
+    MAPREDUCE-3510. Capacity Scheduler inherited ACLs not displayed by mapred queue 
     -showacls (Jonathan Eagles via mahadev)
 
     MAPREDUCE-3537. Fix race condition in DefaultContainerExecutor which led
@@ -4354,7 +4124,13 @@ Release 0.23.1 - 2012-02-17
     in the correct directory to work properly in secure mode. (Hitesh Shah via
     vinodkv)
 
-    MAPREDUCE-3541. Fix broken TestJobQueueClient test. (Ravi Prakash via
+    MAPREDUCE-3544. gridmix build is broken, requires hadoop-archives to be 
+    added as ivy dependency. (tucu)
+
+    MAPREDUCE-3557. MR1 test fail to compile because of missing hadoop-archives 
+    dependency. (tucu)
+
+    MAPREDUCE-3541. Fix broken TestJobQueueClient test. (Ravi Prakash via 
     mahadev)
 
     MAPREDUCE-3398. Fixed log aggregation to work correctly in secure mode.
@@ -4369,7 +4145,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3531. Fixed a race in ContainerTokenSecretManager. (Robert Joseph
     Evans via sseth)
 
-    MAPREDUCE-3560. TestRMNodeTransitions is failing on trunk.
+    MAPREDUCE-3560. TestRMNodeTransitions is failing on trunk. 
     (Siddharth Seth via mahadev)
 
     MAPREDUCE-3487. Fixed JobHistory web-UI to display links to single task's
@@ -4381,14 +4157,14 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3422. Counter display names are not being picked up. (Jonathan
     Eagles via sseth)
 
-    MAPREDUCE-3366. Mapreduce component should use consistent directory structure
+    MAPREDUCE-3366. Mapreduce component should use consistent directory structure 
     layout as HDFS/common (Eric Yang via mahadev)
 
     MAPREDUCE-3387. Fixed AM's tracking URL to always go through the proxy, even
     before the job started, so that it works properly with oozie throughout
     the job execution. (Robert Joseph Evans via vinodkv)
 
-    MAPREDUCE-3579. ConverterUtils shouldn't include a port in a path from a url
+    MAPREDUCE-3579. ConverterUtils shouldn't include a port in a path from a url 
     without a port. (atm via harsh)
 
     MAPREDUCE-3563. Fixed LocalJobRunner to work correctly with new mapreduce
@@ -4403,27 +4179,27 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3588. Fixed bin/yarn which was broken by MAPREDUCE-3366 so that
     yarn daemons can start. (Arun C Murthy via vinodkv)
 
+    MAPREDUCE-3349. Log rack-name in JobHistory for unsuccessful tasks. (Amar
+    Kamat and Devaraj K via sseth)
+
     MAPREDUCE-3586. Modified CompositeService to avoid duplicate stop operations
     thereby solving race conditions in MR AM shutdown. (vinodkv)
 
     MAPREDUCE-3604. Fixed streaming to use new mapreduce.framework.name to
-    check for local mode. (acmurthy)
+    check for local mode. (acmurthy) 
 
     MAPREDUCE-3521. Fixed streaming to ensure it doesn't silently ignore
-    unknown arguments. (Robert Evans via acmurthy)
+    unknown arguments. (Robert Evans via acmurthy) 
 
     MAPREDUCE-3522. Ensure queues inherit ACLs from parent if they aren't
-    explicitly specified. (Jonathan Eagles via acmurthy)
+    explicitly specified. (Jonathan Eagles via acmurthy) 
 
     MAPREDUCE-3608. Fixed compile issue with MAPREDUCE-3522. (mahadev via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-3490. Fixed MapReduce AM to count failed maps also towards Reduce
     ramp up. (Sharad Agarwal and Arun C Murthy via vinodkv)
 
-    MAPREDUCE-1744. DistributedCache creates its own FileSytem instance when
-    adding a file/archive to the path. (Dick King via tucu)
-
     MAPREDUCE-3529. TokenCache does not cache viewfs credentials correctly
     (sseth)
 
@@ -4431,45 +4207,25 @@ Release 0.23.1 - 2012-02-17
     1 to 0.23 (Tom White via sseth)
 
     MAPREDUCE-3566. Fixed MR AM to construct CLC only once across all tasks.
-    (vinodkv via acmurthy)
+    (vinodkv via acmurthy) 
 
     MAPREDUCE-3572. Moved AM event dispatcher to a separate thread for
-    performance reasons. (vinodkv via acmurthy)
+    performance reasons. (vinodkv via acmurthy) 
 
     MAPREDUCE-3615. Fix some ant test failures. (Thomas Graves via sseth)
 
+    MAPREDUCE-1744. DistributedCache creates its own FileSytem instance when 
+    adding a file/archive to the path. (Dick King via tucu)
+
     MAPREDUCE-3326. Added detailed information about queue's to the
-    CapacityScheduler web-ui. (Jason Lowe via acmurthy)
+    CapacityScheduler web-ui. (Jason Lowe via acmurthy) 
 
     MAPREDUCE-3548. Added more unit tests for MR AM & JHS web-services.
-    (Thomas Graves via acmurthy)
+    (Thomas Graves via acmurthy) 
 
     MAPREDUCE-3617. Removed wrong default value for
     yarn.resourcemanager.principal and yarn.nodemanager.principal. (Jonathan
-    Eagles via acmurthy)
-
-    MAPREDUCE-3183. hadoop-assemblies/src/main/resources/assemblies/hadoop-mapreduce-dist.xml
-    missing license header. (Hitesh Shah via tucu).
-
-    MAPREDUCE-3003. Publish MR JARs to Maven snapshot repository. (tucu)
-
-    MAPREDUCE-3204. mvn site:site fails on MapReduce. (tucu)
-
-    MAPREDUCE-3014. Rename and invert logic of '-cbuild' profile to 'native' and off
-    by default. (tucu)
-
-    MAPREDUCE-3477. Hadoop site documentation cannot be built anymore. (jeagles via tucu)
-
-    MAPREDUCE-3500. MRJobConfig creates an LD_LIBRARY_PATH using the platform ARCH. (tucu)
-
-    MAPREDUCE-3389. MRApps loads the 'mrapp-generated-classpath' file with
-    classpath from the build machine. (tucu)
-
-    MAPREDUCE-3544. gridmix build is broken, requires hadoop-archives to be added as
-    ivy dependency. (tucu)
-
-    MAPREDUCE-3557. MR1 test fail to compile because of missing hadoop-archives dependency.
-    (tucu)
+    Eagles via acmurthy) 
 
     MAPREDUCE-3624. Remove unnecessary dependency on JDK's tools.jar. (mahadev
     via acmurthy)
@@ -4480,7 +4236,7 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3639. Fixed TokenCache to work with absent FileSystem canonical
     service-names. (Siddharth Seth via vinodkv)
 
-    MAPREDUCE-3380. Token infrastructure for running clients which are not kerberos
+    MAPREDUCE-3380. Token infrastructure for running clients which are not kerberos 
     authenticated. (mahadev)
 
     MAPREDUCE-3648. TestJobConf failing. (Thomas Graves via mahadev)
@@ -4488,8 +4244,8 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3651. TestQueueManagerRefresh fails. (Thomas Graves via mahadev)
 
     MAPREDUCE-3645. TestJobHistory fails. (Thomas Graves via mahadev)
-
-    MAPREDUCE-3652. org.apache.hadoop.mapred.TestWebUIAuthorization.testWebUIAuthorization
+  
+    MAPREDUCE-3652. org.apache.hadoop.mapred.TestWebUIAuthorization.testWebUIAuthorization 
     fails. (Thomas Graves via mahadev)
 
     MAPREDUCE-3625. CapacityScheduler web-ui display of queue's used capacity is broken.
@@ -4507,10 +4263,13 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3404. Corrected MR AM to honor speculative configuration and enable
     speculating either maps or reduces. (Eric Payne via vinodkv)
 
+    MAPREDUCE-3664. Federation Documentation has incorrect configuration example.
+    (Brandon Li via jitendra)
+
     MAPREDUCE-3649. Job End notification gives an error on calling back.
     (Ravi Prakash via mahadev)
 
-    MAPREDUCE-3657. State machine visualize build fails. (Jason Lowe
+    MAPREDUCE-3657. State machine visualize build fails. (Jason Lowe 
     via mahadev)
 
     MAPREDUCE-2450. Fixed a corner case with interrupted communication threads
@@ -4531,24 +4290,27 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3689. RM web UI doesn't handle newline in job name.
     (Thomas Graves via mahadev)
 
-    MAPREDUCE-3549. write api documentation for web service apis for RM, NM,
+    MAPREDUCE-3701. Delete HadoopYarnRPC from 0.23 branch.
+    (mahadev)
+
+    MAPREDUCE-3549. write api documentation for web service apis for RM, NM, 
     mapreduce app master, and job history server (Thomas Graves via mahadev)
 
     MAPREDUCE-3705. ant build fails on 0.23 branch. (Thomas Graves via
     mahadev)
-
+ 
     MAPREDUCE-3691. webservices add support to compress response.
     (Thomas Graves via mahadev)
 
-    MAPREDUCE-3702. internal server error trying access application master
+    MAPREDUCE-3702. internal server error trying access application master 
     via proxy with filter enabled (Thomas Graves via mahadev)
 
     MAPREDUCE-3646. Remove redundant URL info from "mapred job" output.
     (Jonathan Eagles via mahadev)
 
-    MAPREDUCE-3681. Fixed computation of queue's usedCapacity. (acmurthy)
+    MAPREDUCE-3681. Fixed computation of queue's usedCapacity. (acmurthy) 
 
-    MAPREDUCE-3505. yarn APPLICATION_CLASSPATH needs to be overridable.
+    MAPREDUCE-3505. yarn APPLICATION_CLASSPATH needs to be overridable. 
     (ahmed via tucu)
 
     MAPREDUCE-3714. Fixed EventFetcher and Fetcher threads to shut-down properly
@@ -4571,7 +4333,7 @@ Release 0.23.1 - 2012-02-17
     per-application. (Arun C Murthy via vinodkv)
 
     MAPREDUCE-3721. Fixed a race in shuffle which caused reduces to hang.
-    (sseth via acmurthy)
+    (sseth via acmurthy) 
 
     MAPREDUCE-3733. Add Apache License Header to hadoop-distcp/pom.xml.
     (mahadev)
@@ -4580,7 +4342,7 @@ Release 0.23.1 - 2012-02-17
     (mahadev)
 
     MAPREDUCE-3720. Changed bin/mapred job -list to not print job-specific
-    information not available at RM. (vinodkv via acmurthy)
+    information not available at RM. (vinodkv via acmurthy) 
 
     MAPREDUCE-3742. "yarn logs" command fails with ClassNotFoundException.
     (Jason Lowe via mahadev)
@@ -4593,7 +4355,7 @@ Release 0.23.1 - 2012-02-17
     tasks. (Jonathan Eagles via vinodkv)
 
     MAPREDUCE-3748. Changed a log in CapacityScheduler.nodeUpdate to debug.
-    (ramya via acmurthy)
+    (ramya via acmurthy) 
 
     MAPREDUCE-3764. Fixed resource usage metrics for queues and users.
     (acmurthy)
@@ -4602,16 +4364,16 @@ Release 0.23.1 - 2012-02-17
     (tomwhite)
 
     MAPREDUCE-3762. Fixed default CapacityScheduler configs. (mahadev via
-    acmurthy)
+    acmurthy) 
 
-    MAPREDUCE-3499. New MiniMR does not setup proxyuser configuration
+    MAPREDUCE-3499. New MiniMR does not setup proxyuser configuration 
     correctly, thus tests using doAs do not work. (johnvijoe via tucu)
 
     MAPREDUCE-3696. MR job via oozie does not work on hadoop 23.
     (John George via mahadev)
 
     MAPREDUCE-3427. Fix streaming unit tests broken after mavenization.
-    (Hitesh Shah via acmurthy)
+    (Hitesh Shah via acmurthy) 
 
     MAPREDUCE-3640. Allow AMRecovery to work with partial JobHistory files.
     (Arun C Murthy via sseth)
@@ -4625,10 +4387,10 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3780. Fixed a bug where applications killed before getting
     activated were not getting cleaned up properly. (Hitesh Shah via acmurthy)
 
-    MAPREDUCE-3708. Metrics: Incorrect Apps Submitted Count (Bhallamudi via
+    MAPREDUCE-3708. Metrics: Incorrect Apps Submitted Count (Bhallamudi via 
     mahadev)
 
-    MAPREDUCE-3727. jobtoken location property in jobconf refers to wrong
+    MAPREDUCE-3727. jobtoken location property in jobconf refers to wrong 
     jobtoken file (tucu)
 
     MAPREDUCE-3711. Fixed MR AM recovery so that only single selected task
@@ -4647,14 +4409,14 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3791. can't build site in hadoop-yarn-server-common.
     (mahadev)
 
-    MAPREDUCE-3723. TestAMWebServicesJobs & TestHSWebServicesJobs
+    MAPREDUCE-3723. TestAMWebServicesJobs & TestHSWebServicesJobs 
     incorrectly asserting tests (Bhallamudi Venkata Siva Kamesh
     via mahadev)
 
     MAPREDUCE-3795. "job -status" command line output is malformed.
     (vinodkv via mahadev)
 
-    MAPREDUCE-3759. ClassCastException thrown in -list-active-trackers when
+    MAPREDUCE-3759. ClassCastException thrown in -list-active-trackers when 
     there are a few unhealthy nodes (vinodkv via mahadev)
 
     MAPREDUCE-3775. Change MiniYarnCluster to escape special chars in testname.
@@ -4664,14 +4426,14 @@ Release 0.23.1 - 2012-02-17
     allocation-mb setting (Hitesh Shah via mahadev)
 
     MAPREDUCE-3747. Initialize queue metrics upfront and added start/finish
-    time to RM Web-UI. (acmurthy)
+    time to RM Web-UI. (acmurthy) 
 
     MAPREDUCE-3814. Fixed MRV1 compilation. (Arun C Murthy via vinodkv)
 
     MAPREDUCE-3810. Performance tweaks - reduced logging in AM and defined
-    hascode/equals for ResourceRequest & Priority. (vinodkv via acmurthy)
+    hascode/equals for ResourceRequest & Priority. (vinodkv via acmurthy) 
 
-    MAPREDUCE-3813. Added a cache for resolved racks. (vinodkv via acmurthy)
+    MAPREDUCE-3813. Added a cache for resolved racks. (vinodkv via acmurthy)   
 
     MAPREDUCE-3808. Fixed an NPE in FileOutputCommitter for jobs with maps
     but no reduces. (Robert Joseph Evans via vinodkv)
@@ -4680,24 +4442,21 @@ Release 0.23.1 - 2012-02-17
     (Dave Thompson via bobby)
 
     MAPREDUCE-3354. Changed scripts so that jobhistory server is started by
-    bin/mapred instead of bin/yarn. (Jonathan Eagles via acmurthy)
+    bin/mapred instead of bin/yarn. (Jonathan Eagles via acmurthy) 
 
     MAPREDUCE-3809. Ensure that there is no needless sleep in Task at the end
     of the task. (sseth via acmurthy)
 
-    MAPREDUCE-3794. Support mapred.Task.Counter and mapred.JobInProgress.Counter
+    MAPREDUCE-3794. Support mapred.Task.Counter and mapred.JobInProgress.Counter 
     enums for compatibility (Tom White via mahadev)
 
     MAPREDUCE-3697. Support binary compatibility for Counters after
-    MAPREDUCE-901. (mahadev via acmurthy)
-
-    MAPREDUCE-3709. TestDistributedShell is failing. (Hitesh Shah via
-    mahadev)
+    MAPREDUCE-901. (mahadev via acmurthy) 
 
     MAPREDUCE-3817. Fixed bin/mapred to allow running of distcp and archive
-    jobs. (Arpit Gupta via acmurthy)
+    jobs. (Arpit Gupta via acmurthy) 
 
-    MAPREDUCE-3709. TestDistributedShell is failing. (Hitesh Shah via
+    MAPREDUCE-3709. TestDistributedShell is failing. (Hitesh Shah via 
     mahadev)
 
     MAPREDUCE-3436. JobHistory webapp address should use the host configured
@@ -4706,9 +4465,9 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3815. Fixed MR AM to always use hostnames and never IPs when
     requesting containers so that scheduler can give off data local containers
     correctly. (Siddarth Seth via vinodkv)
-
+ 
     MAPREDUCE-3833. Fixed a bug in reinitiaziling of queues. (Jason Lowe via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-3826. Fixed a bug in RM web-ui which broke sorting. (Jonathan
     Eagles via acmurthy)
@@ -4717,7 +4476,7 @@ Release 0.23.1 - 2012-02-17
     finishes. (Vinod Kumar Vavilapalli via sseth)
 
     MAPREDUCE-3827. Changed Counters to use ConcurrentSkipListMap for
-    performance. (vinodkv via acmurthy)
+    performance. (vinodkv via acmurthy)  
 
     MAPREDUCE-3822. Changed FS counter computation to use all occurences of
     the same FS scheme, instead of randomly using one. (Mahadev Konar via
@@ -4728,14 +4487,11 @@ Release 0.23.1 - 2012-02-17
     be on the same rack. (Siddarth Seth via vinodkv)
 
     MAPREDUCE-3828. Ensure that urls in single-node mode are correct. (sseth
-    via acmurthy)
+    via acmurthy) 
 
     MAPREDUCE-3770. Zombie.getJobConf() results into NPE. (amarrk)
 
-    MAPREDUCE-3840.  JobEndNotifier doesn't use the proxyToUse during connecting
-    (Ravi Prakash via bobby)
-
-    MAPREDUCE-3843. Job summary log file found missing on the RM host
+    MAPREDUCE-3843. Job summary log file found missing on the RM host 
     (Anupam Seth via tgraves)
 
     MAPREDUCE-3846. Addressed MR AM hanging issues during AM restart and then
@@ -4753,7 +4509,7 @@ Release 0.23.1 - 2012-02-17
 
     MAPREDUCE-3880. Changed LCE binary to be 32-bit. (acmurthy)
 
-Release 0.23.0 - 2011-11-01
+Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES
 
@@ -4767,12 +4523,12 @@ Release 0.23.0 - 2011-11-01
   NEW FEATURES
 
     MAPREDUCE-2682. Add "mapred classpath" command to print classpath
-    for MR applications. (vinodkv via acmurthy)
+    for MR applications. (vinodkv via acmurthy) 
 
     MAPREDUCE-2107. [Gridmix] Total heap usage emulation in Gridmix.
     (Amar Kamat and Ravi Gummadi via amarrk)
 
-    MAPREDUCE-2106. [Gridmix] Cumulative CPU usage emulation in Gridmix.
+    MAPREDUCE-2106. [Gridmix] Cumulative CPU usage emulation in Gridmix. 
     (amarrk)
 
     MAPREDUCE-2543. [Gridmix] High-Ram feature emulation in Gridmix. (amarrk)
@@ -4785,14 +4541,14 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-461. Enable ServicePlugins for the JobTracker.
     (Fredrik Hedberg via tomwhite)
 
-    MAPREDUCE-2521. Create RPM and Debian packages for MapReduce. Changes
+    MAPREDUCE-2521. Create RPM and Debian packages for MapReduce. Changes 
     deployment layout to be consistent across the binary tgz, rpm, and deb.
     (Eric Yang via omalley)
 
     MAPREDUCE-2323. Add metrics to the fair scheduler. (todd)
 
     MAPREDUCE-2037. Capture intermediate progress, CPU and memory usage for
-    tasks. (Dick King via acmurthy)
+    tasks. (Dick King via acmurthy) 
 
     MAPREDUCE-279. MapReduce 2.0. Merging MR-279 branch into trunk. Contributed by
     Arun C Murthy, Christopher Douglas, Devaraj Das, Greg Roelofs, Jeffrey
@@ -4812,55 +4568,55 @@ Release 0.23.0 - 2011-11-01
     make MR AMs resume their progress after restart. (Sharad Agarwal via vinodkv)
 
     MAPREDUCE-2858. Added a WebApp Proxy for applications. (Robert Evans via
-    acmurthy)
+    acmurthy) 
 
   IMPROVEMENTS
 
     MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via
-    acmurthy)
+    acmurthy) 
 
-    MAPREDUCE-2365. Add counters to track bytes (read,written) via
+    MAPREDUCE-2365. Add counters to track bytes (read,written) via 
     File(Input,Output)Format. (Siddharth Seth via acmurthy)
-
-    MAPREDUCE-2680. Display queue name in job client CLI. (acmurthy)
-
-    MAPREDUCE-2679. Minor changes to sync trunk with MR-279 branch. (acmurthy)
-
-    MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a
-    ServiceProvider for the actual implementation. (tomwhite via acmurthy)
-
+ 
+    MAPREDUCE-2680. Display queue name in job client CLI. (acmurthy) 
+ 
+    MAPREDUCE-2679. Minor changes to sync trunk with MR-279 branch. (acmurthy) 
+ 
+    MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a 
+    ServiceProvider for the actual implementation. (tomwhite via acmurthy) 
+ 
     MAPREDUCE-2596. [Gridmix] Summarize Gridmix runs. (amarrk)
 
-    MAPREDUCE-2563. [Gridmix] Add High-Ram emulation system tests to
+    MAPREDUCE-2563. [Gridmix] Add High-Ram emulation system tests to 
     Gridmix. (Vinay Kumar Thota via amarrk)
 
-    MAPREDUCE-2104. [Rumen] Add Cpu, Memory and Heap usages to
+    MAPREDUCE-2104. [Rumen] Add Cpu, Memory and Heap usages to 
     TraceBuilder's output. (amarrk)
 
-    MAPREDUCE-2554. [Gridmix]  Add distributed cache emulation system tests
+    MAPREDUCE-2554. [Gridmix]  Add distributed cache emulation system tests 
     to Gridmix. (Vinay Kumar Thota via amarrk)
-
+ 
     MAPREDUCE-2543. [Gridmix] High-Ram feature emulation testcase. (amarrk)
 
-    MAPREDUCE-2469. Task counters should also report the total heap usage of
+    MAPREDUCE-2469. Task counters should also report the total heap usage of 
     the task. (Ravi Gummadi and Amar Ramesh Kamat via amarrk)
 
-    MAPREDUCE-2544. [Gridmix] Add compression emulation system tests to
+    MAPREDUCE-2544. [Gridmix] Add compression emulation system tests to 
     Gridmix. (Vinay Kumar Thota via amarrk)
 
-    MAPREDUCE-2517. [Gridmix] Add system tests to Gridmix.
+    MAPREDUCE-2517. [Gridmix] Add system tests to Gridmix. 
     (Vinay Kumar Thota via amarrk)
 
     MAPREDUCE-2492. The new MapReduce API should make available task's
     progress to the task. (amarrk)
 
-    MAPREDUCE-2153. Bring in more job configuration properties in to the trace
+    MAPREDUCE-2153. Bring in more job configuration properties in to the trace 
     file. (Rajesh Balamohan via amarrk)
 
-    MAPREDUCE-1461. Feature to instruct rumen-folder utility to skip jobs worth
+    MAPREDUCE-1461. Feature to instruct rumen-folder utility to skip jobs worth 
     of specific duration. (Rajesh Balamohan via amarrk)
 
-    MAPREDUCE-2172. Added test-patch.properties required by test-patch.sh
+    MAPREDUCE-2172. Added test-patch.properties required by test-patch.sh 
     (nigel)
 
     MAPREDUCE-2156. Raid-aware FSCK. (Patrick Kling via dhruba)
@@ -4919,12 +4675,12 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2202. Generalize CLITest structure and interfaces to faciliate
     upstream adoption (e.g. for web or system testing). (cos)
 
-    MAPREDUCE-2420. JobTracker should be able to renew delegation token over
+    MAPREDUCE-2420. JobTracker should be able to renew delegation token over 
     HTTP (Boris Shkolnik via jitendra)
 
     MAPREDUCE-2474. Add docs to the new API Partitioner on how to access the
     Job Configuration. (Harsh J Chouraria via todd)
-
+    
     MAPREDUCE-2475. Disable IPV6 for junit tests. (suresh srinivas via mahadev)
 
     MAPREDUCE-2422. Removed unused internal methods from DistributedCache.
@@ -4970,7 +4726,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2494. Order distributed cache deletions by LRU. (Robert Joseph
     Evans via cdouglas)
 
-    MAPREDUCE-2452. Makes the cancellation of delegation tokens happen in a
+    MAPREDUCE-2452. Makes the cancellation of delegation tokens happen in a 
     separate thread. (ddas)
 
     HADOOP-7106. Reorganize project SVN layout to "unsplit" the projects.
@@ -4985,10 +4741,10 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2602. Allow setting of end-of-record delimiter for
     TextInputFormat for the old API. (Ahmed Radwan via todd)
 
-    MAPREDUCE-2705. Permits parallel multiple task launches.
+    MAPREDUCE-2705. Permits parallel multiple task launches. 
     (Thomas Graves via ddas)
 
-    MAPREDUCE-2489. Jobsplits with random hostnames can make the queue
+    MAPREDUCE-2489. Jobsplits with random hostnames can make the queue 
     unusable (jeffrey naisbit via mahadev)
 
     MAPREDUCE-2854. update INSTALL with config necessary run mapred on yarn.
@@ -5002,29 +4758,29 @@ Release 0.23.0 - 2011-11-01
     vinodkv)
 
     MAPREDUCE-2735. Add an applications summary log to ResourceManager.
-    (Thomas Graves via acmurthy)
+    (Thomas Graves via acmurthy) 
 
     MAPREDUCE-2697. Enhance CapacityScheduler to cap concurrently running
-    applications per-queue & per-user. (acmurthy)
+    applications per-queue & per-user. (acmurthy) 
     Configuration changes:
       add yarn.capacity-scheduler.maximum-am-resource-percent
 
     MAPREDUCE-2774. Add startup message to ResourceManager & NodeManager on
-    startup. (Venu Gopala Rao via acmurthy)
+    startup. (Venu Gopala Rao via acmurthy) 
 
     MAPREDUCE-2655. Add audit logs to ResourceManager and NodeManager. (Thomas
     Graves via acmurthy)
 
     MAPREDUCE-2864. Normalize configuration variable names for YARN. (Robert
-    Evans via acmurthy)
+    Evans via acmurthy) 
 
-    MAPREDUCE-2690. Web-page for FifoScheduler. (Eric Payne via acmurthy)
+    MAPREDUCE-2690. Web-page for FifoScheduler. (Eric Payne via acmurthy) 
 
     MAPREDUCE-2711. Update TestBlockPlacementPolicyRaid for the new namesystem
     and block management APIs.  (szetszwo)
 
     MAPREDUCE-2933. Change allocate call to return ContainerStatus for
-    completed containers rather than Container. (acmurthy)
+    completed containers rather than Container. (acmurthy) 
 
     MAPREDUCE-2675. Reformat JobHistory Server main page to be more
     useful. (Robert Joseph Evans via vinodkv).
@@ -5033,16 +4789,16 @@ Release 0.23.0 - 2011-11-01
     org.apache.hadoop.yarn.api.records.* to be get/set only. Added javadocs to
     all public records. (acmurthy)
 
-    MAPREDUCE-2676. MR-279: JobHistory Job page needs reformatted. (Robert Evans via
+    MAPREDUCE-2676. MR-279: JobHistory Job page needs reformatted. (Robert Evans via 
     mahadev)
 
-    MAPREDUCE-2899. Replace major parts of ApplicationSubmissionContext with a
+    MAPREDUCE-2899. Replace major parts of ApplicationSubmissionContext with a 
     ContainerLaunchContext (Arun Murthy via mahadev)
 
     MAPREDUCE-2966. Added ShutDown hooks for MRV2 processes so that they can
     gracefully exit. (Abhijit Suresh Shingate via vinodkv)
 
-    MAPREDUCE-2672. MR-279: JobHistory Server needs Analysis this job.
+    MAPREDUCE-2672. MR-279: JobHistory Server needs Analysis this job. 
     (Robert Evans via mahadev)
 
     MAPREDUCE-2965. Streamlined the methods hashCode(), equals(), compareTo()
@@ -5070,9 +4826,9 @@ Release 0.23.0 - 2011-11-01
     application's FinalStatus separately. (Hitesh Shah via vinodkv)
 
     MAPREDUCE-2889. Added documentation for writing new YARN applications.
-    (Hitesh Shah via acmurthy)
+    (Hitesh Shah via acmurthy) 
 
-    MAPREDUCE-3134. Added documentation the CapacityScheduler. (acmurthy)
+    MAPREDUCE-3134. Added documentation the CapacityScheduler. (acmurthy) 
 
     MAPREDUCE-3013. Removed YarnConfiguration.YARN_SECURITY_INFO and its usage
     as it doesn't affect security any more. (vinodkv)
@@ -5088,21 +4844,21 @@ Release 0.23.0 - 2011-11-01
     UI. (Robert Joseph Evans via vinodkv)
 
     MAPREDUCE-2988. Reenabled TestLinuxContainerExecutor reflecting the
-    current NodeManager code. (Robert Joseph Evans via vinodkv)
+    current NodeManager code. (Robert Joseph Evans via vinodkv) 
 
     MAPREDUCE-3161. Improved some javadocs and fixed some typos in
     YARN. (Todd Lipcon via vinodkv)
 
     MAPREDUCE-3148. Ported MAPREDUCE-2702 to old mapred api for aiding task
-    recovery. (acmurthy)
+    recovery. (acmurthy) 
 
     MAPREDUCE-3133. Running a set of methods in a Single Test Class.
     (Jonathan Eagles via mahadev)
 
-    MAPREDUCE-3059. QueueMetrics do not have metrics for aggregate
+    MAPREDUCE-3059. QueueMetrics do not have metrics for aggregate 
     containers-allocated and aggregate containers-released.
     (Devaraj K via mahadev)
-
+   
     MAPREDUCE-3187. Add names for various unnamed threads in MR2.
     (Todd Lipcon and Siddharth Seth via mahadev)
 
@@ -5115,11 +4871,11 @@ Release 0.23.0 - 2011-11-01
 
     MAPREDUCE-3144. Augmented JobHistory with the information needed for
     serving aggregated logs. (Siddharth Seth via vinodkv)
-
+  
     MAPREDUCE-3163. JobClient spews errors when killing MR2 job.
     (mahadev)
 
-    MAPREDUCE-3239. Use new createSocketAddr API in MRv2 to give better
+    MAPREDUCE-3239. Use new createSocketAddr API in MRv2 to give better 
     error messages on misconfig (Todd Lipcon via mahadev)
 
     MAPREDUCE-2747. Cleaned up LinuxContainerExecutor binary sources and changed
@@ -5127,7 +4883,7 @@ Release 0.23.0 - 2011-11-01
 
     MAPREDUCE-3205. Fix memory specifications to be physical rather than
     virtual, allowing for a ratio between the two to be configurable. (todd
-    via acmurthy)
+    via acmurthy) 
 
     MAPREDUCE-2986. Fixed MiniYARNCluster to support multiple NodeManagers.
     (Anupam Seth via vinodkv)
@@ -5137,6 +4893,12 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2989. Modified JobHistory to link to task and AM logs from the
     JobHistoryServer. (Siddharth Seth via vinodkv)
 
+    MAPREDUCE-3014. Rename and invert logic of '-cbuild' profile to 'native' and off 
+    by default. (tucu)
+
+    MAPREDUCE-3171. normalize nodemanager native code compilation with common/hdfs
+    native. (tucu)
+
     MAPREDUCE-3146. Added a MR specific command line to dump logs for a
     given TaskAttemptID. (Siddharth Seth via vinodkv)
 
@@ -5144,7 +4906,7 @@ Release 0.23.0 - 2011-11-01
     acmurthy)
 
     MAPREDUCE-3322. Added a better index.html and an brief overview of YARN
-    architecture. (acmurthy)
+    architecture. (acmurthy) 
 
   OPTIMIZATIONS
 
@@ -5162,11 +4924,11 @@ Release 0.23.0 - 2011-11-01
 
   BUG FIXES
 
-    MAPREDUCE-2603. Disable High-Ram emulation in system tests.
+    MAPREDUCE-2603. Disable High-Ram emulation in system tests. 
     (Vinay Kumar Thota via amarrk)
 
     MAPREDUCE-2539. Fixed NPE in getMapTaskReports in JobClient. (Robert Evans via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-1978. Rumen TraceBuilder should provide recursive
     input folder scanning.
@@ -5185,7 +4947,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-1752. Implement getFileBlockLocations in HarFilesystem.
     (Patrick Kling via dhruba)
 
-    MAPREDUCE-2155. RaidNode should optionally use the mapreduce jobs to
+    MAPREDUCE-2155. RaidNode should optionally use the mapreduce jobs to 
     fix missing blocks.  (Patrick Kling via dhruba)
 
     MAPREDUCE-1334. Fix TestIndexUpdater by ignoring _SUCCESS file in HDFS.
@@ -5221,7 +4983,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-1242. Chain APIs error misleading.
     (Harsh J Chouraria via tomwhite)
 
-    MAPREDUCE-2379. Adds missing DistributedCache configurations in
+    MAPREDUCE-2379. Adds missing DistributedCache configurations in 
     mapred-default.xml (Todd Lipcon via amareshwari)
 
     MAPREDUCE-2348. Disable mumak tests on trunk since they currently time out
@@ -5241,7 +5003,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2451. Log the details from health check script at the
     JobTracker. (Thomas Graves via cdouglas)
 
-    MAPREDUCE-2467. HDFS-1052 changes break the raid contrib module in
+    MAPREDUCE-2467. HDFS-1052 changes break the raid contrib module in 
     MapReduce. (suresh srinivas via mahadev)
 
     MAPREDUCE-2258. IFile reader closes stream and compressor in wrong order.
@@ -5295,7 +5057,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2409. DistributedCache maps files and archives to the same path,
     despite semantic incompatibility. (Siddharth Seth via cdouglas)
 
-    MAPREDUCE-2575. TestMiniMRDFSCaching fails if test.build.dir is set
+    MAPREDUCE-2575. TestMiniMRDFSCaching fails if test.build.dir is set 
     to something other than build/test (Thomas Graves via mahadev)
 
     MAPREDUCE-2622. Remove the last remaining reference to the deprecated
@@ -5320,12 +5082,12 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2797. Update mapreduce tests and RAID for HDFS-2239.  (szetszwo)
 
     MAPREDUCE-2805. Update RAID for HDFS-2241.  (szetszwo)
-
+    
     MAPREDUCE-2837. Ported bug fixes from y-merge to prepare for MAPREDUCE-279
-    merge. (acmurthy)
+    merge. (acmurthy) 
 
     MAPREDUCE-2541. Fixed a race condition in IndexCache.removeMap. (Binglin
-    Chang via acmurthy)
+    Chang via acmurthy) 
 
     MAPREDUCE-2458. Rename sanitized pom.xml in build directory to work around IDE
     bug (Luke Lu via mahadev)
@@ -5382,10 +5144,10 @@ Release 0.23.0 - 2011-11-01
     ApplicationMaster information. (acmurthy)
 
     MAPREDUCE-279. Fix in MR-279 branch. Fixed computation of user-limits at
-    runtime. (acmurthy)
+    runtime. (acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Added functionality to refresh queues at
-    runtime via the 'bin/yarn rmadmin' command. (acmurthy)
+    runtime via the 'bin/yarn rmadmin' command. (acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Added functionality to stop/start queues.
     (acmurthy)
@@ -5406,7 +5168,7 @@ Release 0.23.0 - 2011-11-01
     failures. (sharad)
 
     MAPREDUCE-279. Fix in MR-279 branch. Added support High-RAM applications in
-    CapacityScheduler. (acmurthy)
+    CapacityScheduler. (acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Completing the ZooKeeper Store for
     ResourceManager state. (mahadev)
@@ -5463,7 +5225,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Fix refreshProxy in ClientServiceDelegate.
     (sharad)
 
-    MAPREDUCE-279. Fix in MR-279 branch. Fix Null Pointer in TestUberAM. (sharad)
+    MAPREDUCE-279. Fix in MR-279 branch. Fix Null Pointer in TestUberAM. (sharad) 
 
     MAPREDUCE-2478. Improve history server. (Siddharth Seth via sharad)
 
@@ -5542,7 +5304,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Makes uber-task disabled by default (ddas)
 
     MAPREDUCE-279. Fix in MR-279 branch. Make logging and memory for AM configurable
-    for the user via command line (mahadev)
+    for the user via command line (mahadev) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Fixing a bug in previous patch (r1103657).
     Now bin/yarn truly shouldn't be launched multiple times in a single AM.
@@ -5557,26 +5319,26 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Fix the tests to use jvm fork mode to avoid
     errors in shutting down services (sidharth seth)
 
-    MAPREDUCE-2500. PB factories are not thread safe (Siddharth Seth via mahadev)
+    MAPREDUCE-2500. PB factories are not thread safe (Siddharth Seth via mahadev) 
 
     MAPREDUCE-2504. race in JobHistoryEventHandler stop (Siddharth Seth via mahadev)
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix job hang if the AM launch fails.
-    (mahadev)
+    (mahadev) 
 
-    MAPREDUCE-2509. Fix NPE in UI for pending attempts. (Luke Lu via mahadev)
+    MAPREDUCE-2509. Fix NPE in UI for pending attempts. (Luke Lu via mahadev) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Add junit jar to lib in assembly (mahadev
     and luke)
 
     MAPREDUCE-279. Fix in MR-279 branch. Distributed cache bug fix to pass Terasort.
     (vinodkv)
-
+     
     MAPREDUCE-279. Fix in MR-279 branch. Fix null pointer exception in kill task
     attempt (mahadev)
 
     MAPREDUCE-279. Fix in MR-279 branch. Refactored RMContainerAllocator to release
-    unused containers. (sharad)
+    unused containers. (sharad) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Changed Scheduler to return available limit
     to AM in the allocate api. (acmurthy)
@@ -5615,7 +5377,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Add public cache. (cdouglas)
 
     MAPREDUCE-279. Fix in MR-279 branch. Made number of RPC server threads
-    configurable. (acmurthy)
+    configurable. (acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Added acl check for RMAdmin. (acmurthy)
 
@@ -5674,7 +5436,7 @@ Release 0.23.0 - 2011-11-01
     to get log-locations from an environmental variable. (vinodkv)
 
     MAPREDUCE-279. Fix in MR-279 branch. Ensure 'lost' NodeManagers are dealt
-    appropriately, the containers are released correctly. (acmurthy)
+    appropriately, the containers are released correctly. (acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Adding some more logging for AM expiry logs
     (mahadev)
@@ -5686,7 +5448,7 @@ Release 0.23.0 - 2011-11-01
     tasks. (vinodkv)
 
     MAPREDUCE-279. Fix in MR-279 branch. Added ability to decommission nodes and
-    completed RM administration tools to achieve parity with JobTracker. (acmurthy)
+    completed RM administration tools to achieve parity with JobTracker. (acmurthy) 
 
     MAPREDUCE-2551. Added JobSummaryLog. (Siddharth Seth via acmurthy)
 
@@ -5722,14 +5484,14 @@ Release 0.23.0 - 2011-11-01
     user-logs to a separate hdfs file. (vinodkv)
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix calculation of max-capacity for a
-    queue, also fixed a bug in registration of NodeManagers. (acmurthy)
+    queue, also fixed a bug in registration of NodeManagers. (acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. More cleaning up constants, removing stale
     code, and making conspicuous the envs that apps depend on to be provided by
     YARN. (vinodkv)
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix container size rounding in AM and
-    headroom in RM. (acmurthy and sharad)
+    headroom in RM. (acmurthy and sharad) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Disable Job acls until fixed (mahadev)
 
@@ -5738,7 +5500,7 @@ Release 0.23.0 - 2011-11-01
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix a corner case in headroom computation -
     now reservations are taken into account and headroom is computed much later to
-    account for allocations/reservations. (acmurthy)
+    account for allocations/reservations. (acmurthy) 
 
     MAPREDUCE-2537. The RM writes its log to
     yarn-mapred-resourcemanager-<RM_Host>.out (Robert Evans via mahadev)
@@ -5752,10 +5514,10 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Support for min and max container capacity.
     (acmurthy and sharad)
 
-    MAPREDUCE-2531. Fixed jobcontrol to downgrade JobID. (Robert Evans via acmurthy)
+    MAPREDUCE-2531. Fixed jobcontrol to downgrade JobID. (Robert Evans via acmurthy) 
 
     MAPREDUCE-2539. Fixed NPE in getMapTaskReports in JobClient. (Robert Evans via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Fixing the wrong config key used in
     JobHistory that prevented configuring move-thread interval. (vinodkv)
@@ -5822,7 +5584,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Add ability to includes src files in
     assembly target for maven (Luke Lu via mahadev)
 
-    MAPREDUCE-2582. Cleanup JobHistory event generation.(Siddharth Seth via sharad)
+    MAPREDUCE-2582. Cleanup JobHistory event generation.(Siddharth Seth via sharad)  
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix rounding off problem in reduce ramp up.
     (sharad)
@@ -5951,7 +5713,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2628. Add compiled on date to NM and RM info/about page.
 
     MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a ServiceProvider
-    for the actual implementation. (tomwhite via acmurthy)
+    for the actual implementation. (tomwhite via acmurthy) 
 
     MAPREDUCE-2663. Refactoring StateMachineFactory inner classes. (ahmed radwan via
     mahadev)
@@ -5966,7 +5728,7 @@ Release 0.23.0 - 2011-11-01
     explicitly set in the Configuration. (Josh Wills via vinodkv)
 
     MAPREDUCE-2661. Fix TaskImpl to not access MapTaskImpl. (Ahmed Radwan via
-    sharad)
+    sharad) 
 
     HADOOP-6929. Backport changes to MR-279 (mahadev and owen)
 
@@ -5995,7 +5757,7 @@ Release 0.23.0 - 2011-11-01
     mahadev)
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix findbugs warnings in mr-client modules,
-    part 1 (mahadev)
+    part 1 (mahadev) 
 
     MAPREDUCE-279. Fix in MR-279 branch. Fix findbugs warnings in mr-client modules
     part 2 (mahadev)
@@ -6015,9 +5777,9 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-279. Fix in MR-279 branch. Fix for running ant targets to use the
     right set of common/test jars (gkesavan via mahadev)
 
-    MAPREDUCE-2782. Unit tests for CapacityScheduler. (acmurthy)
+    MAPREDUCE-2782. Unit tests for CapacityScheduler. (acmurthy) 
 
-    MAPREDUCE-2706. Log job submission failures. (Jeffrey Naisbitt via acmurthy)
+    MAPREDUCE-2706. Log job submission failures. (Jeffrey Naisbitt via acmurthy) 
 
     MAPREDUCE-2781. mr279 RM application finishtime not set (Thomas Graves via
     mahadev)
@@ -6040,7 +5802,7 @@ Release 0.23.0 - 2011-11-01
 
     MAPREDUCE-2727. Fix divide-by-zero error in SleepJob for sleepCount equals
     0. (Jeffrey Naisbitt via acmurthy)
-
+ 
     MAPREDUCE-2860. Fix log4j logging in the maven test cases. (mahadev)
 
     MAPREDUCE-2867. Remove Unused TestApplicaitonCleanup in resourcemanager/applicationsmanager.
@@ -6049,41 +5811,41 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2868. ant build broken in hadoop-mapreduce dir (mahadev, giri and arun via mahadev)
 
     MAPREDUCE-2649. Handling of finished applications in RM. (Thomas Graves
-    via acmurthy)
+    via acmurthy) 
 
     MAPREDUCE-2838. Fix MapReduce builds to use new hadoop-common test jars.
-    (gkesavan via acmurthy)
-
-    MAPREDUCE-2859. Fix eclipse plugin contrib module compilation (gkesavan)
+    (gkesavan via acmurthy) 
+   
+    MAPREDUCE-2859. Fix eclipse plugin contrib module compilation (gkesavan) 
 
     MAPREDUCE-2846. Fix missing synchronization in the task log management.
     (omalley)
 
     MAPREDUCE-2807. Fix AM restart and client redirection. (sharad)
 
-    MAPREDUCE-2877. Add missing Apache license header in some files in MR
+    MAPREDUCE-2877. Add missing Apache license header in some files in MR 
     and also add the rat plugin to the poms. (mahadev)
 
     MAPREDUCE-2796. Set start times for MR applications for clients to see.
-    (Devaraj K via acmurthy)
-
-    MAPREDUCE-2879. Fix version for MR-279 to 0.23.0. (acmurthy)
+    (Devaraj K via acmurthy) 
 
+    MAPREDUCE-2879. Fix version for MR-279 to 0.23.0. (acmurthy) 
+   
     MAPREDUCE-2881. Fix to include log4j 1.2.16 depenency (gkesavan)
 
     MAPREDUCE-2885. Fix mapred-config.sh to look for hadoop-config.sh in
-    HADOOP_COMMON_HOME/libexec. (acmurthy)
+    HADOOP_COMMON_HOME/libexec. (acmurthy) 
 
     MAPREDUCE-2893. Remove duplicate entry of YarnClientProtocolProvider in
-    ClientProtocolProvider services file. (Liang-Chi Hsieh via acmurthy)
+    ClientProtocolProvider services file. (Liang-Chi Hsieh via acmurthy) 
 
     MAPREDUCE-2891. Javadoc for AMRMProtocol and related records. (acmurthy)
 
-    MAPREDUCE-2898. Javadoc for ContainerManager protocol and related records.
+    MAPREDUCE-2898. Javadoc for ContainerManager protocol and related records. 
     (acmurthy)
 
     MAPREDUCE-2904. Fixed bin/yarn to correctly include HDFS jars and
-    clean up of stale refs to pre-mavenized Hadoop Common and HDFS.
+    clean up of stale refs to pre-mavenized Hadoop Common and HDFS. 
     (Sharad Agarwal and Arun C. Murthy via acmurthy)
 
     MAPREDUCE-2737. Update the progress of jobs on client side. (Siddharth Seth
@@ -6091,17 +5853,17 @@ Release 0.23.0 - 2011-11-01
 
     MAPREDUCE-2886. Fix Javadoc warnings in MapReduce. (mahadev)
 
-    MAPREDUCE-2897. Javadoc for ClientRMProtocol protocol and related records.
+    MAPREDUCE-2897. Javadoc for ClientRMProtocol protocol and related records. 
     (acmurthy)
 
-    MAPREDUCE-2916. Ivy build for MRv1 fails with bad organization for
+    MAPREDUCE-2916. Ivy build for MRv1 fails with bad organization for 
     common daemon. (mahadev)
 
     MAPREDUCE-2917. Fixed corner case in container reservation which led to
-    starvation and hung jobs. (acmurthy)
+    starvation and hung jobs. (acmurthy) 
 
     MAPREDUCE-2756. Better error handling in JobControl for failed jobs.
-    (Robert Evans via acmurthy)
+    (Robert Evans via acmurthy) 
 
     MAPREDUCE-2716. MRReliabilityTest job fails because of missing
     job-file. (Jeffrey Naisbitt via vinodkv)
@@ -6109,42 +5871,42 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2882. TestLineRecordReader depends on ant jars. (todd)
 
     MAPREDUCE-2687. Fix NodeManager to use the right version of
-    LocalDirAllocator.getLocalPathToWrite. (mahadev & acmurthy)
+    LocalDirAllocator.getLocalPathToWrite. (mahadev & acmurthy) 
 
     MAPREDUCE-2800. Set final progress for tasks to ensure all task information
     is correctly logged to JobHistory. (Siddharth Seth via acmurthy)
 
     MAPREDUCE-2938. Log application submission failure in CapacityScheduler.
-    (acmurthy)
+    (acmurthy) 
 
     MAPREDUCE-2948. Hadoop streaming test failure, post MR-2767 (mahadev)
 
-    MAPREDUCE-2908. Fix all findbugs warnings. (vinodkv via acmurthy)
+    MAPREDUCE-2908. Fix all findbugs warnings. (vinodkv via acmurthy) 
 
     MAPREDUCE-2947. Fixed race condition in AuxiliaryServices. (vinodkv via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-2844. Fixed display of nodes in UI. (Ravi Teja Ch N V via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-2677. Fixed 404 for some links from HistoryServer. (Robert Evans
-    via acmurthy)
+    via acmurthy) 
 
     MAPREDUCE-2937. Ensure reason for application failure is displayed to the
-    user. (mahadev via acmurthy)
+    user. (mahadev via acmurthy) 
 
-    MAPREDUCE-2953. Fix a race condition on submission which caused client to
+    MAPREDUCE-2953. Fix a race condition on submission which caused client to 
     incorrectly assume application was gone by making submission synchronous
-    for RMAppManager. (Thomas Graves via acmurthy)
+    for RMAppManager. (Thomas Graves via acmurthy) 
 
-    MAPREDUCE-2963. Fix hang in TestMRJobs. (Siddharth Seth via acmurthy)
+    MAPREDUCE-2963. Fix hang in TestMRJobs. (Siddharth Seth via acmurthy) 
 
     MAPREDUCE-2954. Fixed a deadlock in NM caused due to wrong synchronization
     in protocol buffer records. (Siddharth Seth via vinodkv)
 
     MAPREDUCE-2975. Fixed YARNRunner to use YarnConfiguration rather than
-    Configuration. (mahadev via acmurthy)
-
+    Configuration. (mahadev via acmurthy) 
+ 
     MAPREDUCE-2971. ant build mapreduce fails protected access jc.displayJobList
     (jobs) (Thomas Graves via mahadev)
 
@@ -6164,18 +5926,18 @@ Release 0.23.0 - 2011-11-01
     acmurthy)
 
     MAPREDUCE-2995. Better handling of expired containers in MapReduce
-    ApplicationMaster. (vinodkv via acmurthy)
+    ApplicationMaster. (vinodkv via acmurthy) 
 
-    MAPREDUCE-2995. Fixed race condition in ContainerLauncher. (vinodkv via
-    acmurthy)
+    MAPREDUCE-2995. Fixed race condition in ContainerLauncher. (vinodkv via 
+    acmurthy) 
 
     MAPREDUCE-2949. Fixed NodeManager to shut-down correctly if a service
     startup fails. (Ravi Teja via vinodkv)
 
     MAPREDUCE-3005. Fix both FifoScheduler and CapacityScheduler to correctly
-    enforce locality constraints. (acmurthy)
+    enforce locality constraints. (acmurthy) 
 
-    MAPREDUCE-3007. Fixed Yarn Mapreduce client to be able to connect to
+    MAPREDUCE-3007. Fixed Yarn Mapreduce client to be able to connect to 
     JobHistoryServer in secure mode. (vinodkv)
 
     MAPREDUCE-2987. Fixed display of logged user on RM Web-UI. (Thomas Graves
@@ -6191,7 +5953,7 @@ Release 0.23.0 - 2011-11-01
     reject all NMs. (Devaraj K via vinodkv)
 
     MAPREDUCE-3042. Fixed default ResourceTracker address. (Chris Riccomini
-    via acmurthy)
+    via acmurthy) 
 
     MAPREDUCE-3038. job history server not starting because conf() missing
     HsController (Jeffrey Naisbitt via mahadev)
@@ -6204,9 +5966,9 @@ Release 0.23.0 - 2011-11-01
 
     MAPREDUCE-3040. Fixed extra copy of Configuration in
     YarnClientProtocolProvider and ensured MiniMRYarnCluster sets JobHistory
-    configuration for tests. (acmurthy)
+    configuration for tests. (acmurthy) 
 
-    MAPREDUCE-3018. Fixed -file option for streaming. (mahadev via acmurthy)
+    MAPREDUCE-3018. Fixed -file option for streaming. (mahadev via acmurthy) 
 
     MAPREDUCE-3036. Fixed metrics for reserved resources in CS. (Robert Evans
     via acmurthy)
@@ -6215,16 +5977,16 @@ Release 0.23.0 - 2011-11-01
     bin/mapred too many times. (vinodkv via acmurthy)
 
     MAPREDUCE-3023. Fixed clients to display queue state correctly. (Ravi
-    Prakash via acmurthy)
+    Prakash via acmurthy) 
 
     MAPREDUCE-2970. Fixed NPEs in corner cases with different configurations
     for mapreduce.framework.name. (Venu Gopala Rao via vinodkv)
 
     MAPREDUCE-3062. Fixed default RMAdmin address. (Chris Riccomini
-    via acmurthy)
+    via acmurthy) 
 
-    MAPREDUCE-3066. Fixed default ResourceTracker address for the NodeManager.
-    (Chris Riccomini via acmurthy)
+    MAPREDUCE-3066. Fixed default ResourceTracker address for the NodeManager. 
+    (Chris Riccomini via acmurthy) 
 
     MAPREDUCE-3044. Pipes jobs stuck without making progress. (mahadev)
 
@@ -6238,7 +6000,7 @@ Release 0.23.0 - 2011-11-01
 
 
     MAPREDUCE-2990. Fixed display of NodeHealthStatus. (Subroto Sanyal via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-3053. Better diagnostic message for unknown methods in ProtoBuf
     RPCs. (vinodkv via acmurthy)
@@ -6246,12 +6008,12 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2952. Fixed ResourceManager/MR-client to consume diagnostics
     for AM failures in a couple of corner cases. (Arun C Murthy via vinodkv)
 
-    MAPREDUCE-3064. 27 unit test failures with Invalid
-    "mapreduce.jobtracker.address" configuration value for
+    MAPREDUCE-3064. 27 unit test failures with Invalid 
+    "mapreduce.jobtracker.address" configuration value for 
     JobTracker: "local" (Venu Gopala Rao via mahadev)
 
     MAPREDUCE-3090. Fix MR AM to use ApplicationAttemptId rather than
-    (ApplicationId, startCount) consistently. (acmurthy)
+    (ApplicationId, startCount) consistently. (acmurthy)  
 
     MAPREDUCE-2646. Fixed AMRMProtocol to return containers based on
     priority. (Sharad Agarwal and Arun C Murthy via vinodkv)
@@ -6262,7 +6024,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2984. Better error message for displaying completed containers.
     (Devaraj K via acmurthy)
 
-    MAPREDUCE-3071. app master configuration web UI link under the Job menu
+    MAPREDUCE-3071. app master configuration web UI link under the Job menu 
     opens up application menu. (thomas graves  via mahadev)
 
     MAPREDUCE-3067. Ensure exit-code is set correctly for containers. (Hitesh
@@ -6277,7 +6039,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-3054. Unable to kill submitted jobs. (mahadev)
 
     MAPREDUCE-3021. Change base urls for RM web-ui. (Thomas Graves via
-    acmurthy)
+    acmurthy) 
 
     MAPREDUCE-3041. Fixed ClientRMProtocol to provide min/max resource
     capabilities along-with new ApplicationId for application submission.
@@ -6301,13 +6063,16 @@ Release 0.23.0 - 2011-11-01
     via acmurthy)
 
     MAPREDUCE-3050. Add ability to get resource usage information for
-    applications and nodes. (Robert Evans via acmurthy)
+    applications and nodes. (Robert Evans via acmurthy) 
 
     MAPREDUCE-3113. Ensure bin/yarn and bin/yarn-daemon.sh identify the root
-    of the install properly. (Xie Xianshan via acmurthy)
+    of the install properly. (Xie Xianshan via acmurthy) 
+
+    MAPREDUCE-3137. Fix broken merge of MAPREDUCE-2179. (Hitesh Shah via
+    acmurthy) 
 
     MAPREDUCE-2792. Replace usage of node ip-addresses with hostnames.
-    (vinodkv via acmurthy)
+    (vinodkv via acmurthy) 
 
     MAPREDUCE-3112. Fixed recursive sourcing of HADOOP_OPTS environment
     variable. (Eric Yang)
@@ -6318,7 +6083,7 @@ Release 0.23.0 - 2011-11-01
     MAPREDUCE-2913. Fixed TestMRJobs.testFailingMapper to assert the correct
     TaskCompletionEventStatus. (Jonathan Eagles via vinodkv)
 
-    MAPREDUCE-2794. [MR-279] Incorrect metrics value for AvailableGB per
+    MAPREDUCE-2794. [MR-279] Incorrect metrics value for AvailableGB per 
     queue per user. (John George via mahadev)
 
     MAPREDUCE-2783. Fixing RM web-UI to show no tracking-URL when AM
@@ -6334,19 +6099,19 @@ Release 0.23.0 - 2011-11-01
     frameworks. (Hitesh Shah via acmurthy)
 
     MAPREDUCE-2802. Ensure JobHistory filenames have jobId. (Jonathan Eagles
-    via acmurthy)
+    via acmurthy) 
 
     MAPREDUCE-2876. Use a different config for ContainerAllocationExpirer.
-    (Anupam Seth via acmurthy)
+    (Anupam Seth via acmurthy) 
 
     MAPREDUCE-3153. Fix TestFileOutputCommitter which was broken by
-    MAPREDUCE-2702. (mahadev via acmurthy)
+    MAPREDUCE-2

<TRUNCATED>

[09/37] hadoop git commit: MAPREDUCE-6566. Add retry support to mapreduce CLI tool. Contributed by Varun Vasudev

Posted by as...@apache.org.
MAPREDUCE-6566. Add retry support to mapreduce CLI tool. Contributed by Varun Vasudev


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fc470840
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fc470840
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fc470840

Branch: refs/heads/yarn-2877
Commit: fc470840a0b1f5dc8b3b13f7ed99fd68ba728216
Parents: 7fa9ea8
Author: Xuan <xg...@apache.org>
Authored: Mon Dec 7 14:15:14 2015 -0800
Committer: Xuan <xg...@apache.org>
Committed: Mon Dec 7 14:15:14 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            | 1395 ++++++++----------
 .../org/apache/hadoop/mapreduce/tools/CLI.java  |   41 +-
 .../apache/hadoop/mapreduce/tools/TestCLI.java  |   31 +-
 3 files changed, 646 insertions(+), 821 deletions(-)
----------------------------------------------------------------------



[10/37] hadoop git commit: Update CHANGES.txt for commit of YARN-4348 to branch-2.7 and branch-2.6.

Posted by as...@apache.org.
Update CHANGES.txt for commit of YARN-4348 to branch-2.7 and branch-2.6.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d7b3f8db
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d7b3f8db
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d7b3f8db

Branch: refs/heads/yarn-2877
Commit: d7b3f8dbe818cff5fee4f4c0c70d306776aa318e
Parents: fc47084
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Dec 9 00:03:20 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Dec 9 00:03:20 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7b3f8db/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3ba94b4..dcd9ec5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -2029,6 +2029,9 @@ Release 2.6.3 - UNRELEASED
     YARN-4365. FileSystemNodeLabelStore should check for root dir existence on
     startup (Kuhu Shukla via jlowe)
 
+    YARN-4348. ZKRMStateStore.syncInternal shouldn't wait for sync completion for
+    avoiding blocking ZK's event thread. (ozawa)
+
 Release 2.6.2 - 2015-10-28
 
   INCOMPATIBLE CHANGES


[37/37] hadoop git commit: YARN-4358 addendum patch to fix javadoc error

Posted by as...@apache.org.
YARN-4358 addendum patch to fix javadoc error


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fb212e5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fb212e5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fb212e5

Branch: refs/heads/yarn-2877
Commit: 7fb212e5e672b3f285493f6c7b2598ef7e61406b
Parents: bf5295b
Author: Arun Suresh <as...@apache.org>
Authored: Sat Dec 12 22:22:55 2015 -0800
Committer: Arun Suresh <as...@apache.org>
Committed: Sat Dec 12 22:22:55 2015 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/reservation/PlanView.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fb212e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index f57c2e0..cf00a92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -47,7 +47,8 @@ public interface PlanView extends PlanContext {
    *
    * @param user the user being considered
    * @param t the instant in time being considered
-   * @return {@link Set<ReservationAllocation>} for this user at this time
+   * @return set of active {@link ReservationAllocation}s for this
+   *         user at this time
    */
   public Set<ReservationAllocation> getReservationByUserAtTime(String user,
       long t);


[30/37] hadoop git commit: YARN-4341. add doc about timeline performance tool usage (Chang Li via sjlee)

Posted by as...@apache.org.
YARN-4341. add doc about timeline performance tool usage (Chang Li via sjlee)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e8964589
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e8964589
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e8964589

Branch: refs/heads/yarn-2877
Commit: e8964589de27a507c31203a87c3cbe0a8d68d3dd
Parents: a483017
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Dec 11 09:46:32 2015 -0800
Committer: Sangjin Lee <sj...@apache.org>
Committed: Fri Dec 11 09:46:32 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../src/site/markdown/TimelineServer.md         | 78 ++++++++++++++++++++
 2 files changed, 81 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8964589/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 32d76f4..a29194a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -607,6 +607,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4248. REST API for submit/update/delete Reservations. (curino)
 
+    YARN-4341. add doc about timeline performance tool usage (Chang Li via
+    sjlee)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8964589/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
index 2048012..3e589d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -26,6 +26,10 @@ The YARN Timeline Server
 * [Publishing of application specific data](#Publishing_of_application_specific_data)
 * [Timeline Server REST API](#Timeline_Server_REST_API_v1)
 * [Generic Data REST APIs](#GENERIC_DATA_REST_APIS)
+* [Timelnine Server Performance Test Tool](#TIMELINE_SERVER_PERFORMANCE_TEST_TOOL)
+    * [Highlights](#HIGHLIGHTS)
+    * [Usage](#USAGE)
+    * [Sample Runs](#SAMPLE_RUNS)
 
 <a name="Overview"></a>Overview
 ---------
@@ -2033,3 +2037,77 @@ This hides details of other domains from an unauthorized caller.
 this failure *will not* result in an HTTP error code being retured.
 A status code of 200 will be returned —however, there will be an error code
 in the list of failed entities for each entity which could not be added.
+
+<a name="TIMELINE_SERVER_PERFORMANCE_TEST_TOOL"></a> Timelnine Server Performance Test Tool
+----------
+###<a name="HIGHLIGHTS"></a>Highlights
+
+The timeline server performance test tool helps measure timeline server's write performance. The test
+launches SimpleEntityWriter mappers or JobHistoryFileReplay mappers to write timeline
+entities to the timeline server. At the end, the transaction rate(ops/s) per mapper and the total transaction rate
+will be measured and printed out. Running the test with SimpleEntityWriter mappers
+will also measure and show the IO rate(KB/s) per mapper and the total IO rate.
+
+###<a name="USAGE"></a>Usage
+
+Mapper Types Description:
+
+     1. SimpleEntityWriter mapper
+        Each mapper writes a user-specified number of timeline entities
+        with a user-specified size to the timeline server.
+
+     2. JobHistoryFileReplay mapper
+        Each mapper replays jobhistory files under a specified directory
+        (both the jhist file and its corresponding conf.xml are required to
+         be present in order to be replayed. The number of mappers should be no more
+         than the number of jobhistory files).
+        Each mapper will get assigned some jobhistory files to replay. For each
+        job history file, a mapper will parse it to get jobinfo and then create
+        timeline entities. Each mapper also has the choice to write all the
+        timeline entities created at once or one at a time.
+
+Options:
+
+    [-m <maps>] number of mappers (default: 1)
+    [-v] timeline service version
+    [-mtype <mapper type in integer>]
+          1. simple entity write mapper
+          2. jobhistory files replay mapper
+    [-s <(KBs)test>] number of KB per put (mtype=1, default: 1 KB)
+    [-t] package sending iterations per mapper (mtype=1, default: 100)
+    [-d <path>] root path of job history files (mtype=2)
+    [-r <replay mode>] (mtype=2)
+          1. write all entities for a job in one put (default)
+          2. write one entity at a time
+
+###<a name="SAMPLE_RUNS"></a>Sample Runs
+
+Run SimpleEntityWriter test:
+
+    bin/hadoop jar performanceTest.jar timelineperformance -m 4 -mtype 1 -s 3 -t 200
+
+Example output of SimpleEntityWriter test :
+
+    TRANSACTION RATE (per mapper): 20000.0 ops/s
+    IO RATE (per mapper): 60000.0 KB/s
+    TRANSACTION RATE (total): 80000.0 ops/s
+    IO RATE (total): 240000.0 KB/s
+
+Run JobHistoryFileReplay mapper test
+
+    $ bin/hadoop jar performanceTest.jar timelineperformance -m 2 -mtype 2 -d /testInput -r 2
+
+Example input of JobHistoryFileReplay mapper test:
+
+    $ bin/hadoop fs -ls /testInput
+    /testInput/job_1.jhist
+    /testInput/job_1_conf.xml
+    /testInput/job_2.jhist
+    /testInput/job_2_conf.xml
+
+Eample output of JobHistoryFileReplay test:
+
+    TRANSACTION RATE (per mapper): 4000.0 ops/s
+    IO RATE (per mapper): 0.0 KB/s
+    TRANSACTION RATE (total): 8000.0 ops/s
+    IO RATE (total): 0.0 KB/s


[33/37] hadoop git commit: HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)

Posted by as...@apache.org.
HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/796a676d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/796a676d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/796a676d

Branch: refs/heads/yarn-2877
Commit: 796a676d18bd7cd3ed4113d002e0e69cf261d6d1
Parents: 576b569
Author: Uma Mahesh <um...@apache.org>
Authored: Fri Dec 11 17:57:35 2015 -0800
Committer: Uma Mahesh <um...@apache.org>
Committed: Fri Dec 11 17:57:35 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/blockmanagement/BlockManager.java    |  51 +++-------
 .../blockmanagement/DecommissionManager.java    |  32 +++---
 .../hdfs/server/namenode/FSNamesystem.java      | 102 +++++++++----------
 .../hdfs/server/namenode/NamenodeFsck.java      |  10 +-
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |  18 ++--
 .../apache/hadoop/hdfs/TestFileCreation.java    |   6 +-
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 8 files changed, 89 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index f84e1e5..09f08e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1737,6 +1737,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-9472. concat() API does not give proper exception messages on ./reserved 
     relative path (Rakesh R via umamahesh)
 
+    HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 197850a..5efad7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1072,27 +1072,17 @@ public class BlockManager implements BlockStatsMXBean {
    public void verifyReplication(String src,
                           short replication,
                           String clientName) throws IOException {
+    String err = null;
+    if (replication > maxReplication) {
+      err = " exceeds maximum of " + maxReplication;
+    } else if (replication < minReplication) {
+      err = " is less than the required minimum of " + minReplication;
+    }
 
-    if (replication < minReplication || replication > maxReplication) {
-      StringBuilder msg = new StringBuilder("Requested replication factor of ");
-
-      msg.append(replication);
-
-      if (replication > maxReplication) {
-        msg.append(" exceeds maximum of ");
-        msg.append(maxReplication);
-      } else {
-        msg.append(" is less than the required minimum of ");
-        msg.append(minReplication);
-      }
-
-      msg.append(" for ").append(src);
-
-      if (clientName != null) {
-        msg.append(" from ").append(clientName);
-      }
-
-      throw new IOException(msg.toString());
+    if (err != null) {
+      throw new IOException("Requested replication factor of " + replication
+          + err + " for " + src
+          + (clientName == null? "": ", clientName=" + clientName));
     }
   }
 
@@ -1228,8 +1218,7 @@ public class BlockManager implements BlockStatsMXBean {
       }
     }
     if (datanodes.length() != 0) {
-      blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock,
-          datanodes.toString());
+      blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock, datanodes);
     }
   }
 
@@ -2850,7 +2839,8 @@ public class BlockManager implements BlockStatsMXBean {
     if (result == AddBlockResult.ADDED) {
       curReplicaDelta = 1;
       if (logEveryBlock) {
-        logAddStoredBlock(storedBlock, node);
+        blockLog.debug("BLOCK* addStoredBlock: {} is added to {} (size={})",
+            node, storedBlock, storedBlock.getNumBytes());
       }
     } else if (result == AddBlockResult.REPLACED) {
       curReplicaDelta = 0;
@@ -2923,21 +2913,6 @@ public class BlockManager implements BlockStatsMXBean {
     return storedBlock;
   }
 
-  private void logAddStoredBlock(BlockInfo storedBlock,
-      DatanodeDescriptor node) {
-    if (!blockLog.isDebugEnabled()) {
-      return;
-    }
-
-    StringBuilder sb = new StringBuilder(500);
-    sb.append("BLOCK* addStoredBlock: blockMap updated: ")
-      .append(node)
-      .append(" is added to ");
-    storedBlock.appendStringTo(sb);
-    sb.append(" size " )
-      .append(storedBlock.getNumBytes());
-    blockLog.debug(sb.toString());
-  }
   /**
    * Invalidate corrupt replicas.
    * <p>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index b9b5bf4..8656f69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.util.Time.monotonicNow;
+
 import java.util.AbstractList;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -26,13 +29,9 @@ import java.util.Queue;
 import java.util.TreeMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -44,8 +43,9 @@ import org.apache.hadoop.util.ChunkedArrayList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.hadoop.util.Time.monotonicNow;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * Manages datanode decommissioning. A background monitor thread 
@@ -441,17 +441,10 @@ public class DecommissionManager {
             LOG.debug("Node {} is sufficiently replicated and healthy, "
                 + "marked as decommissioned.", dn);
           } else {
-            if (LOG.isDebugEnabled()) {
-              StringBuilder b = new StringBuilder("Node {} ");
-              if (isHealthy) {
-                b.append("is ");
-              } else {
-                b.append("isn't ");
-              }
-              b.append("healthy and still needs to replicate {} more blocks," +
-                  " decommissioning is still in progress.");
-              LOG.debug(b.toString(), dn, blocks.size());
-            }
+            LOG.debug("Node {} {} healthy."
+                + " It needs to replicate {} more blocks."
+                + " Decommissioning is still in progress.",
+                dn, isHealthy? "is": "isn't", blocks.size());
           }
         } else {
           LOG.debug("Node {} still has {} blocks to replicate "
@@ -587,8 +580,7 @@ public class DecommissionManager {
   }
 
   @VisibleForTesting
-  void runMonitor() throws ExecutionException, InterruptedException {
-    Future f = executor.submit(monitor);
-    f.get();
+  void runMonitorForTest() throws ExecutionException, InterruptedException {
+    executor.submit(monitor).get();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 7e52625..ba6f0e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -317,19 +317,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   NameNodeMXBean {
   public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
 
-  private static final ThreadLocal<StringBuilder> auditBuffer =
-    new ThreadLocal<StringBuilder>() {
-      @Override
-      protected StringBuilder initialValue() {
-        return new StringBuilder();
-      }
-  };
-
   private final BlockIdManager blockIdManager;
 
-  @VisibleForTesting
-  public boolean isAuditEnabled() {
-    return !isDefaultAuditLogger || auditLog.isInfoEnabled();
+  boolean isAuditEnabled() {
+    return (!isDefaultAuditLogger || auditLog.isInfoEnabled())
+        && !auditLoggers.isEmpty();
   }
 
   private void logAuditEvent(boolean succeeded, String cmd, String src)
@@ -358,14 +350,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
           stat.getGroup(), symlink, path);
     }
+    final String ugiStr = ugi.toString();
     for (AuditLogger logger : auditLoggers) {
       if (logger instanceof HdfsAuditLogger) {
         HdfsAuditLogger hdfsLogger = (HdfsAuditLogger) logger;
-        hdfsLogger.logAuditEvent(succeeded, ugi.toString(), addr, cmd, src, dst,
+        hdfsLogger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst,
             status, CallerContext.getCurrent(), ugi, dtSecretManager);
       } else {
-        logger.logAuditEvent(succeeded, ugi.toString(), addr,
-            cmd, src, dst, status);
+        logger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst, status);
       }
     }
   }
@@ -616,15 +608,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       // sharedEditsDirs.
       if (!editsDirs.contains(u) &&
           !sharedEditsDirs.contains(u)) {
-        throw new IllegalArgumentException(
-            "Required edits directory " + u.toString() + " not present in " +
-            DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + ". " +
-            DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" +
-            editsDirs.toString() + "; " +
-            DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY + "=" +
-            requiredEditsDirs.toString() + ". " +
-            DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY + "=" +
-            sharedEditsDirs.toString() + ".");
+        throw new IllegalArgumentException("Required edits directory " + u
+            + " not found: "
+            + DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" + editsDirs + "; "
+            + DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY
+            + "=" + requiredEditsDirs + "; "
+            + DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY
+            + "=" + sharedEditsDirs);
       }
     }
 
@@ -698,11 +688,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
       throws IOException {
     provider = DFSUtil.createKeyProviderCryptoExtension(conf);
-    if (provider == null) {
-      LOG.info("No KeyProvider found.");
-    } else {
-      LOG.info("Found KeyProvider: " + provider.toString());
-    }
+    LOG.info("KeyProvider: " + provider);
     if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY,
                         DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
       LOG.info("Enabling async auditlog");
@@ -2077,11 +2063,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           .append(", clientMachine=").append(clientMachine)
           .append(", createParent=").append(createParent)
           .append(", replication=").append(replication)
-          .append(", createFlag=").append(flag.toString())
+          .append(", createFlag=").append(flag)
           .append(", blockSize=").append(blockSize)
           .append(", supportedVersions=")
-          .append(supportedVersions == null ? null : Arrays.toString
-              (supportedVersions));
+          .append(Arrays.toString(supportedVersions));
       NameNode.stateChangeLog.debug(builder.toString());
     }
     if (!DFSUtil.isValidName(src)) {
@@ -2513,43 +2498,40 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
   }
 
-  INodeFile checkLease(
-      String src, String holder, INode inode, long fileId)
+  private String leaseExceptionString(String src, long fileId, String holder) {
+    final Lease lease = leaseManager.getLease(holder);
+    return src + " (inode " + fileId + ") " + (lease != null? lease.toString()
+        : "Holder " + holder + " does not have any open files.");
+  }
+
+  INodeFile checkLease(String src, String holder, INode inode, long fileId)
       throws LeaseExpiredException, FileNotFoundException {
     assert hasReadLock();
-    final String ident = src + " (inode " + fileId + ")";
     if (inode == null) {
-      Lease lease = leaseManager.getLease(holder);
-      throw new FileNotFoundException(
-          "No lease on " + ident + ": File does not exist. "
-          + (lease != null ? lease.toString()
-              : "Holder " + holder + " does not have any open files."));
+      throw new FileNotFoundException("File does not exist: "
+          + leaseExceptionString(src, fileId, holder));
     }
     if (!inode.isFile()) {
-      Lease lease = leaseManager.getLease(holder);
-      throw new LeaseExpiredException(
-          "No lease on " + ident + ": INode is not a regular file. "
-              + (lease != null ? lease.toString()
-              : "Holder " + holder + " does not have any open files."));
+      throw new LeaseExpiredException("INode is not a regular file: "
+          + leaseExceptionString(src, fileId, holder));
     }
     final INodeFile file = inode.asFile();
     if (!file.isUnderConstruction()) {
-      Lease lease = leaseManager.getLease(holder);
-      throw new LeaseExpiredException(
-          "No lease on " + ident + ": File is not open for writing. "
-          + (lease != null ? lease.toString()
-              : "Holder " + holder + " does not have any open files."));
+      throw new LeaseExpiredException("File is not open for writing: "
+          + leaseExceptionString(src, fileId, holder));
     }
     // No further modification is allowed on a deleted file.
     // A file is considered deleted, if it is not in the inodeMap or is marked
     // as deleted in the snapshot feature.
     if (isFileDeleted(file)) {
-      throw new FileNotFoundException(src);
+      throw new FileNotFoundException("File is deleted: "
+          + leaseExceptionString(src, fileId, holder));
     }
-    String clientName = file.getFileUnderConstructionFeature().getClientName();
-    if (holder != null && !clientName.equals(holder)) {
-      throw new LeaseExpiredException("Lease mismatch on " + ident +
-          " owned by " + clientName + " but is accessed by " + holder);
+    final String owner = file.getFileUnderConstructionFeature().getClientName();
+    if (holder != null && !owner.equals(holder)) {
+      throw new LeaseExpiredException("Client (=" + holder
+          + ") is not the lease owner (=" + owner + ": "
+          + leaseExceptionString(src, fileId, holder));
     }
     return file;
   }
@@ -6188,7 +6170,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (success) {
         getEditLog().logSync();
       }
-      String idStr = "{id: " + directive.getId().toString() + "}";
+      final String idStr = "{id: " + directive.getId() + "}";
       logAuditEvent(success, "modifyCacheDirective", idStr,
           directive.toString(), null);
     }
@@ -6662,6 +6644,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   @VisibleForTesting
   static class DefaultAuditLogger extends HdfsAuditLogger {
+    private static final ThreadLocal<StringBuilder> STRING_BUILDER =
+        new ThreadLocal<StringBuilder>() {
+          @Override
+          protected StringBuilder initialValue() {
+            return new StringBuilder();
+          }
+        };
+
     private boolean isCallerContextEnabled;
     private int callerContextMaxLen;
     private int callerSignatureMaxLen;
@@ -6696,7 +6686,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       if (auditLog.isDebugEnabled() ||
           (auditLog.isInfoEnabled() && !debugCmdSet.contains(cmd))) {
-        final StringBuilder sb = auditBuffer.get();
+        final StringBuilder sb = STRING_BUILDER.get();
         sb.setLength(0);
         sb.append("allowed=").append(succeeded).append("\t");
         sb.append("ugi=").append(userName).append("\t");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index fec2abd..4181996 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -320,7 +320,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         sb.append("FSCK started by " +
             UserGroupInformation.getCurrentUser() + " from " +
             remoteAddress + " at " + new Date());
-        out.println(sb.toString());
+        out.println(sb);
         sb.append(" for blockIds: \n");
         for (String blk: blocks) {
           if(blk == null || !blk.contains(Block.BLOCK_FILE_PREFIX)) {
@@ -331,7 +331,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           blockIdCK(blk);
           sb.append(blk + "\n");
         }
-        LOG.info(sb.toString());
+        LOG.info(sb);
         namenode.getNamesystem().logFsckEvent("/", remoteAddress);
         out.flush();
         return;
@@ -383,7 +383,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         out.println(ecRes);
 
         if (this.showStoragePolcies) {
-          out.print(storageTypeSummary.toString());
+          out.print(storageTypeSummary);
         }
 
         out.println("FSCK ended at " + new Date() + " in "
@@ -725,7 +725,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         // might still be accessible as the block might be incorrectly marked as
         // corrupted by client machines.
         report.append(" MISSING!");
-        res.addMissing(block.toString(), block.getNumBytes());
+        res.addMissing(blkName, block.getNumBytes());
         missing++;
         missize += block.getNumBytes();
       } else {
@@ -793,7 +793,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         out.print(" OK\n");
       }
       if (showBlocks) {
-        out.print(report.toString() + "\n");
+        out.print(report + "\n");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index 1acab73..62f6cd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
@@ -198,14 +199,12 @@ public class TestFileAppend4 {
       t.join();
       LOG.info("Close finished.");
  
-      // We expect that close will get a "File is not open"
-      // error.
+      // We expect that close will get a "File is not open" error.
       Throwable thrownByClose = err.get();
       assertNotNull(thrownByClose);
-      assertTrue(thrownByClose instanceof IOException);
-      if (!thrownByClose.getMessage().contains(
-            "No lease on /testRecoverFinalized"))
-        throw thrownByClose;
+      assertTrue(thrownByClose instanceof LeaseExpiredException);
+      GenericTestUtils.assertExceptionContains("File is not open for writing",
+          thrownByClose);
     } finally {
       cluster.shutdown();
     }
@@ -281,10 +280,9 @@ public class TestFileAppend4 {
       // error.
       Throwable thrownByClose = err.get();
       assertNotNull(thrownByClose);
-      assertTrue(thrownByClose instanceof IOException);
-      if (!thrownByClose.getMessage().contains(
-            "Lease mismatch"))
-        throw thrownByClose;
+      assertTrue(thrownByClose instanceof LeaseExpiredException);
+      GenericTestUtils.assertExceptionContains("not the lease owner",
+          thrownByClose);
       
       // The appender should be able to close properly
       appenderStream.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index 09df4bd..05c98ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -165,7 +165,7 @@ public class TestFileCreation {
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     try {
-      FsServerDefaults serverDefaults = fs.getServerDefaults();
+      FsServerDefaults serverDefaults = fs.getServerDefaults(new Path("/"));
       assertEquals(DFS_BLOCK_SIZE_DEFAULT, serverDefaults.getBlockSize());
       assertEquals(DFS_BYTES_PER_CHECKSUM_DEFAULT, serverDefaults.getBytesPerChecksum());
       assertEquals(DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT, serverDefaults.getWritePacketSize());
@@ -413,8 +413,7 @@ public class TestFileCreation {
         stm1.close();
         fail("Should have exception closing stm1 since it was deleted");
       } catch (IOException ioe) {
-        GenericTestUtils.assertExceptionContains("No lease on /testfile", ioe);
-        GenericTestUtils.assertExceptionContains("File does not exist.", ioe);
+        GenericTestUtils.assertExceptionContains("File does not exist", ioe);
       }
       
     } finally {
@@ -805,7 +804,6 @@ public class TestFileCreation {
   public static void testFileCreationNonRecursive(FileSystem fs) throws IOException {
     final Path path = new Path("/" + Time.now()
         + "-testFileCreationNonRecursive");
-    FSDataOutputStream out = null;
     IOException expectedException = null;
     final String nonExistDir = "/non-exist-" + Time.now();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/796a676d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index a828e64..7ea78c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -306,7 +306,7 @@ public class BlockManagerTestUtil {
    */
   public static void recheckDecommissionState(DatanodeManager dm)
       throws ExecutionException, InterruptedException {
-    dm.getDecomManager().runMonitor();
+    dm.getDecomManager().runMonitorForTest();
   }
 
   /**


[34/37] hadoop git commit: HDFS-9532. Detailed exception info is lost in reportTo methods of ErrorReportAction and ReportBadBlockAction. (Yongjun Zhang)

Posted by as...@apache.org.
HDFS-9532. Detailed exception info is lost in reportTo methods of ErrorReportAction and ReportBadBlockAction. (Yongjun Zhang)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f5a91145
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f5a91145
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f5a91145

Branch: refs/heads/yarn-2877
Commit: f5a911452f8f6bc8a37617d1fc155e5ec23f3a78
Parents: 796a676
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Dec 11 17:33:55 2015 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Dec 11 18:15:24 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 .../hdfs/server/datanode/BPServiceActorActionException.java     | 5 +++++
 .../apache/hadoop/hdfs/server/datanode/ErrorReportAction.java   | 2 +-
 .../hadoop/hdfs/server/datanode/ReportBadBlockAction.java       | 2 +-
 4 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5a91145/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 09f08e9..1fda1d78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1739,6 +1739,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)
 
+    HDFS-9532. Detailed exception info is lost in reportTo methods of
+    ErrorReportAction and ReportBadBlockAction. (Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5a91145/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActorActionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActorActionException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActorActionException.java
index 2028252..389ff26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActorActionException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActorActionException.java
@@ -30,4 +30,9 @@ public class BPServiceActorActionException extends IOException {
   public BPServiceActorActionException(String message) {
     super(message);
   }
+
+  public BPServiceActorActionException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5a91145/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java
index b7a9dae..26498d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java
@@ -49,7 +49,7 @@ public class ErrorReportAction implements BPServiceActorAction {
           + "errorMessage: " + errorMessage + "  errorCode: " + errorCode, re);
     } catch(IOException e) {
       throw new BPServiceActorActionException("Error reporting "
-          + "an error to namenode: ");
+          + "an error to namenode.", e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5a91145/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
index 671a1fe..be46707 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java
@@ -65,7 +65,7 @@ public class ReportBadBlockAction implements BPServiceActorAction {
           + "block:  " + block , re);
     } catch (IOException e) {
       throw new BPServiceActorActionException("Failed to report bad block "
-          + block + " to namenode: ");
+          + block + " to namenode.", e);
     }
   }
 


[16/37] hadoop git commit: YARN-4424. Fix deadlock in RMAppImpl. (Jian he via wangda)

Posted by as...@apache.org.
YARN-4424. Fix deadlock in RMAppImpl. (Jian he via wangda)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7e471518
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7e471518
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7e471518

Branch: refs/heads/yarn-2877
Commit: 7e4715186d31ac889fba26d453feedcebb11fc70
Parents: c4084d9
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Dec 8 14:25:16 2015 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Dec 8 14:25:16 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 ++
 .../server/resourcemanager/rmapp/RMAppImpl.java | 22 +++++++-------------
 2 files changed, 10 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e471518/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index dcd9ec5..c7eae9f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -2032,6 +2032,8 @@ Release 2.6.3 - UNRELEASED
     YARN-4348. ZKRMStateStore.syncInternal shouldn't wait for sync completion for
     avoiding blocking ZK's event thread. (ozawa)
 
+    YARN-4424. Fix deadlock in RMAppImpl. (Jian he via wangda)
+
 Release 2.6.2 - 2015-10-28
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e471518/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 8baddee..3a9df67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -490,21 +490,15 @@ public class RMAppImpl implements RMApp, Recoverable {
 
   @Override
   public FinalApplicationStatus getFinalApplicationStatus() {
-    this.readLock.lock();
-    try {
-      // finish state is obtained based on the state machine's current state 
-      // as a fall-back in case the application has not been unregistered 
-      // ( or if the app never unregistered itself )
-      // when the report is requested
-      if (currentAttempt != null 
-          && currentAttempt.getFinalApplicationStatus() != null) {
-        return currentAttempt.getFinalApplicationStatus();   
-      }
-      return 
-          createFinalApplicationStatus(this.stateMachine.getCurrentState());
-    } finally {
-      this.readLock.unlock();
+    // finish state is obtained based on the state machine's current state
+    // as a fall-back in case the application has not been unregistered
+    // ( or if the app never unregistered itself )
+    // when the report is requested
+    if (currentAttempt != null
+        && currentAttempt.getFinalApplicationStatus() != null) {
+      return currentAttempt.getFinalApplicationStatus();
     }
+    return createFinalApplicationStatus(this.stateMachine.getCurrentState());
   }
 
   @Override


[21/37] hadoop git commit: YARN-4434. NodeManager Disk Checker parameter documentation is not correct. Contributed by Weiwei Yang.

Posted by as...@apache.org.
YARN-4434. NodeManager Disk Checker parameter documentation is not correct. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/50edcb94
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/50edcb94
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/50edcb94

Branch: refs/heads/yarn-2877
Commit: 50edcb947ccbb736924c43735d23f3c156961049
Parents: ad53c52
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Dec 9 16:23:58 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Dec 9 16:32:33 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50edcb94/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2e0514b..29b9570 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -2040,6 +2040,9 @@ Release 2.6.3 - UNRELEASED
 
     YARN-4424. Fix deadlock in RMAppImpl. (Jian he via wangda)
 
+    YARN-4434. NodeManager Disk Checker parameter documentation is not correct.
+    (Weiwei Yang via aajisaka)
+
 Release 2.6.2 - 2015-10-28
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50edcb94/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
index 4724ea6..b3a92dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
@@ -45,7 +45,7 @@ The following configuration parameters can be used to modify the disk checks:
 | `yarn.nodemanager.disk-health-checker.enable` | true, false | Enable or disable the disk health checker service |
 | `yarn.nodemanager.disk-health-checker.interval-ms` | Positive integer | The interval, in milliseconds, at which the disk checker should run; the default value is 2 minutes |
 | `yarn.nodemanager.disk-health-checker.min-healthy-disks` | Float between 0-1 | The minimum fraction of disks that must pass the check for the NodeManager to mark the node as healthy; the default is 0.25 |
-| `yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage` | Float between 0-100 | The maximum percentage of disk space that may be utilized before a disk is marked as unhealthy by the disk checker service. This check is run for every disk used by the NodeManager. The default value is 100 i.e. the entire disk can be used. |
+| `yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage` | Float between 0-100 | The maximum percentage of disk space that may be utilized before a disk is marked as unhealthy by the disk checker service. This check is run for every disk used by the NodeManager. The default value is 90 i.e. 90% of the disk can be used. |
 | `yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb` | Integer | The minimum amount of free space that must be available on the disk for the disk checker service to mark the disk as healthy. This check is run for every disk used by the NodeManager. The default value is 0 i.e. the entire disk can be used. |
 
 ###External Health Script


[23/37] hadoop git commit: HDFS-9527. The return type of FSNamesystem.getBlockCollection should be changed to INodeFile.

Posted by as...@apache.org.
HDFS-9527. The return type of FSNamesystem.getBlockCollection should be changed to INodeFile.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/132478e8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/132478e8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/132478e8

Branch: refs/heads/yarn-2877
Commit: 132478e805ba0f955345217b8ad87c2d17cccb2d
Parents: e27fffd
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Dec 9 17:55:28 2015 -0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Wed Dec 9 17:55:28 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/blockmanagement/BlockManager.java    |  4 +--
 .../blockmanagement/DecommissionManager.java    |  2 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 36 +++++++++++---------
 .../hdfs/server/namenode/NamenodeFsck.java      |  6 ++--
 .../hadoop/hdfs/server/namenode/Namesystem.java |  3 +-
 .../TestClientProtocolForPipelineRecovery.java  |  4 ++-
 .../blockmanagement/TestBlockManager.java       | 22 ++++++------
 .../blockmanagement/TestReplicationPolicy.java  | 14 +++++---
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  6 ++--
 .../hdfs/server/namenode/TestINodeFile.java     |  5 +++
 11 files changed, 59 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8a892d5..6755f00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1736,6 +1736,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9214. Support reconfiguring dfs.datanode.balance.max.concurrent.moves
     without DN restart. (Xiaobing Zhou via Arpit Agarwal)
 
+    HDFS-9527. The return type of FSNamesystem.getBlockCollection should be
+    changed to INodeFile. (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 8c94c03..197850a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2405,7 +2405,7 @@ public class BlockManager implements BlockStatsMXBean {
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
-        if (namesystem.isInSnapshot(storedBlock)) {
+        if (namesystem.isInSnapshot(storedBlock.getBlockCollectionId())) {
           int numOfReplicas = storedBlock.getUnderConstructionFeature()
               .getNumExpectedLocations();
           bmSafeMode.incrementSafeBlockCount(numOfReplicas, storedBlock);
@@ -3997,7 +3997,7 @@ public class BlockManager implements BlockStatsMXBean {
     return addBlockCollection(block, bc);
   }
 
-  public BlockCollection getBlockCollection(BlockInfo b) {
+  BlockCollection getBlockCollection(BlockInfo b) {
     return namesystem.getBlockCollection(b.getBlockCollectionId());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 42810350..b9b5bf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -533,7 +533,7 @@ public class DecommissionManager {
           continue;
         }
 
-        BlockCollection bc = namesystem.getBlockCollection(bcId);
+        final BlockCollection bc = blockManager.getBlockCollection(block);
         final NumberReplicas num = blockManager.countNodes(block);
         final int liveReplicas = num.liveReplicas();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 9c9d9f5..7e52625 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3172,11 +3172,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   
   @Override
-  public boolean isInSnapshot(BlockInfo blockUC) {
+  public boolean isInSnapshot(long blockCollectionID) {
     assert hasReadLock();
-    final BlockCollection bc = blockManager.getBlockCollection(blockUC);
-    if (bc == null || !(bc instanceof INodeFile)
-        || !bc.isUnderConstruction()) {
+    final INodeFile bc = getBlockCollection(blockCollectionID);
+    if (bc == null || !bc.isUnderConstruction()) {
       return false;
     }
 
@@ -3203,8 +3202,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return true;
   }
 
+  INodeFile getBlockCollection(BlockInfo b) {
+    return getBlockCollection(b.getBlockCollectionId());
+  }
+
   @Override
-  public BlockCollection getBlockCollection(long id) {
+  public INodeFile getBlockCollection(long id) {
     INode inode = getFSDirectory().getInode(id);
     return inode == null ? null : inode.asFile();
   }
@@ -3265,8 +3268,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + " is null, likely because the file owning this block was"
             + " deleted and the block removal is delayed");
       }
-      long bcId = storedBlock.getBlockCollectionId();
-      INodeFile iFile = ((INode)getBlockCollection(bcId)).asFile();
+      final INodeFile iFile = getBlockCollection(storedBlock);
       src = iFile.getFullPathName();
       if (isFileDeleted(iFile)) {
         throw new FileNotFoundException("File not found: "
@@ -3682,8 +3684,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         while (it.hasNext()) {
           Block b = it.next();
           BlockInfo blockInfo = blockManager.getStoredBlock(b);
-          BlockCollection bc = getBlockCollection(
-              blockInfo.getBlockCollectionId());
+          BlockCollection bc = getBlockCollection(blockInfo);
           if (bc.getStoragePolicyID() == lpPolicy.getId()) {
             filesToDelete.add(bc);
           }
@@ -4606,15 +4607,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     
     // check stored block state
     BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
-    if (storedBlock == null || 
-        storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
-        throw new IOException(block + 
-            " does not exist or is not under Construction" + storedBlock);
+    if (storedBlock == null) {
+      throw new IOException(block + " does not exist.");
+    }
+    if (storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
+      throw new IOException("Unexpected BlockUCState: " + block
+          + " is " + storedBlock.getBlockUCState()
+          + " but not " + BlockUCState.UNDER_CONSTRUCTION);
     }
     
     // check file inode
-    long bcId = storedBlock.getBlockCollectionId();
-    final INodeFile file = ((INode)getBlockCollection(bcId)).asFile();
+    final INodeFile file = getBlockCollection(storedBlock);
     if (file == null || !file.isUnderConstruction() || isFileDeleted(file)) {
       throw new IOException("The file " + storedBlock + 
           " belonged to does not exist or it is not under construction.");
@@ -4887,8 +4890,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       while (blkIterator.hasNext()) {
         BlockInfo blk = blkIterator.next();
-        BlockCollection bc = getBlockCollection(blk.getBlockCollectionId());
-        final INode inode = (INode)bc;
+        final INodeFile inode = getBlockCollection(blk);
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
           String src = FSDirectory.getFullPathName(inode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 9d4edb5..fec2abd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -40,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -64,7 +64,6 @@ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -261,8 +260,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);
         return;
       }
-      BlockCollection bc = blockManager.getBlockCollection(blockInfo);
-      INode iNode = (INode) bc;
+      final INodeFile iNode = namenode.getNamesystem().getBlockCollection(blockInfo);
       NumberReplicas numberReplicas= blockManager.countNodes(blockInfo);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 59ad092..f2cc75b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
@@ -62,7 +61,7 @@ public interface Namesystem extends RwLock, SafeMode {
   ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
       throws IOException;
 
-  boolean isInSnapshot(BlockInfo blockUC);
+  boolean isInSnapshot(long blockCollectionID);
 
   CacheManager getCacheManager();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
index 77cfb7c..22009fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
@@ -62,7 +63,8 @@ public class TestClientProtocolForPipelineRecovery {
         namenode.updateBlockForPipeline(firstBlock, "");
         Assert.fail("Can not get a new GS from a finalized block");
       } catch (IOException e) {
-        Assert.assertTrue(e.getMessage().contains("is not under Construction"));
+        Assert.assertTrue(e.getMessage().contains(
+            "not " + BlockUCState.UNDER_CONSTRUCTION));
       }
       
       // test getNewStampAndToken on a non-existent block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 9b7ba4a..5df73ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -51,16 +51,18 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.TestINodeFile;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetworkTopology;
-import org.junit.Assert;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -438,13 +440,12 @@ public class TestBlockManager {
   
   private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
     long inodeId = ++mockINodeId;
-    BlockCollection bc = Mockito.mock(BlockCollection.class);
-    Mockito.doReturn(inodeId).when(bc).getId();
-    Mockito.doReturn(bc).when(fsn).getBlockCollection(inodeId);
-    BlockInfo blockInfo = blockOnNodes(blockId, nodes);
+    final INodeFile bc = TestINodeFile.createINodeFile(inodeId);
 
+    BlockInfo blockInfo = blockOnNodes(blockId, nodes);
     blockInfo.setReplication((short) 3);
     blockInfo.setBlockCollectionId(inodeId);
+    Mockito.doReturn(bc).when(fsn).getBlockCollection(inodeId);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
   }
@@ -747,12 +748,11 @@ public class TestBlockManager {
 
   private BlockInfo addBlockToBM(long blkId) {
     Block block = new Block(blkId);
-    BlockInfo blockInfo =
-        new BlockInfoContiguous(block, (short) 3);
-    BlockCollection bc = Mockito.mock(BlockCollection.class);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
     long inodeId = ++mockINodeId;
-    doReturn(inodeId).when(bc).getId();
+    final INodeFile bc = TestINodeFile.createINodeFile(inodeId);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
+    blockInfo.setBlockCollectionId(inodeId);
     doReturn(bc).when(fsn).getBlockCollection(inodeId);
     return blockInfo;
   }
@@ -761,9 +761,9 @@ public class TestBlockManager {
     Block block = new Block(blkId);
     BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
     blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null);
-    BlockCollection bc = Mockito.mock(BlockCollection.class);
     long inodeId = ++mockINodeId;
-    doReturn(inodeId).when(bc).getId();
+    final INodeFile bc = TestINodeFile.createINodeFile(inodeId);
+    blockInfo.setBlockCollectionId(inodeId);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     doReturn(bc).when(fsn).getBlockCollection(inodeId);
     return blockInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 1a8a088..3493c14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -56,7 +56,10 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.namenode.TestINodeFile;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.net.Node;
 import org.apache.log4j.Level;
@@ -1309,7 +1312,7 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
   @Test(timeout = 60000)
   public void testAddStoredBlockDoesNotCauseSkippedReplication()
       throws IOException {
-    Namesystem mockNS = mock(Namesystem.class);
+    FSNamesystem mockNS = mock(FSNamesystem.class);
     when(mockNS.hasWriteLock()).thenReturn(true);
     when(mockNS.hasReadLock()).thenReturn(true);
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
@@ -1337,10 +1340,11 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     // queue.
     BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
-    BlockCollection bc = mock(BlockCollection.class);
-    when(bc.getId()).thenReturn(1000L);
-    when(mockNS.getBlockCollection(1000L)).thenReturn(bc);
-    bm.addBlockCollection(info, bc);
+    info.setBlockCollectionId(1000L);
+
+    final INodeFile file = TestINodeFile.createINodeFile(1000L);
+    when(mockNS.getBlockCollection(1000L)).thenReturn(file);
+    bm.addBlockCollection(info, file);
 
     // Adding this block will increase its current replication, and that will
     // remove it from the queue.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 9b06f85..cc74190 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -913,7 +913,7 @@ public class TestFsck {
       try {
         fsn.writeLock();
         BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
-        bc = bm.getBlockCollection(bi);
+        bc = fsn.getBlockCollection(bi);
       } finally {
         fsn.writeUnlock();
       }
@@ -1484,7 +1484,7 @@ public class TestFsck {
       try {
         fsn.writeLock();
         BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
-        bc = bm.getBlockCollection(bi);
+        bc = fsn.getBlockCollection(bi);
       } finally {
         fsn.writeUnlock();
       }
@@ -1699,7 +1699,7 @@ public class TestFsck {
       try {
         fsn.writeLock();
         BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
-        bc = bm.getBlockCollection(bi);
+        bc = fsn.getBlockCollection(bi);
       } finally {
         fsn.writeUnlock();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/132478e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index c33e668..89b2854 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -89,6 +89,11 @@ public class TestINodeFile {
   private short replication;
   private long preferredBlockSize = 1024;
 
+  static public INodeFile createINodeFile(long id) {
+    return new INodeFile(id, ("file" + id).getBytes(), perm, 0L, 0L, null,
+        (short)3, 1024L);
+  }
+
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, replication, preferredBlockSize);


[11/37] hadoop git commit: HDFS-9414. Refactor reconfiguration of ClientDatanodeProtocol for reusability. Contributed by Xiaobing Zhou.

Posted by as...@apache.org.
HDFS-9414. Refactor reconfiguration of ClientDatanodeProtocol for reusability. Contributed by Xiaobing Zhou.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/303f80e6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/303f80e6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/303f80e6

Branch: refs/heads/yarn-2877
Commit: 303f80e6567023985f00e5f33ff41a7bbb4d6072
Parents: d7b3f8d
Author: cnauroth <cn...@apache.org>
Authored: Tue Dec 8 10:49:32 2015 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Tue Dec 8 10:49:32 2015 -0800

----------------------------------------------------------------------
 .../hdfs/protocol/ReconfigurationProtocol.java    |  2 +-
 .../protocolPB/ReconfigurationProtocolPB.java     | 14 +++++++-------
 .../protocolPB/ReconfigurationProtocolUtils.java  |  2 +-
 .../hadoop/hdfs/protocolPB/package-info.java      | 18 ++++++++++++++++++
 .../ReconfigurationProtocolServerSideUtils.java   |  2 +-
 .../hadoop/hdfs/server/datanode/DataNode.java     |  2 +-
 6 files changed, 29 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/303f80e6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
index a22192c..75dc877 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReconfigurationProtocol.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.conf.ReconfigurationTaskStatus;
 @InterfaceStability.Evolving
 public interface ReconfigurationProtocol {
 
-  long versionID = 1L;
+  long VERSIONID = 1L;
 
   /**
    * Asynchronously reload configuration on disk and apply changes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303f80e6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolPB.java
index 659edac..95f05b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolPB.java
@@ -24,13 +24,6 @@ import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.ReconfigurationProtocolService;
 
-@KerberosInfo(serverPrincipal =
-    CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
-@ProtocolInfo(
-    protocolName = "org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol",
-    protocolVersion = 1)
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
 /**
  * Protocol that clients use to communicate with the NN/DN to do
  * reconfiguration on the fly.
@@ -38,6 +31,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.Recon
  * Note: This extends the protocolbuffer service based interface to
  * add annotations required for security.
  */
+@KerberosInfo(serverPrincipal =
+    CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.hdfs.protocol.ReconfigurationProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public interface ReconfigurationProtocolPB extends
     ReconfigurationProtocolService.BlockingInterface {
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303f80e6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolUtils.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolUtils.java
index a3fead9..71195c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolUtils.java
@@ -32,7 +32,7 @@ import com.google.common.collect.Maps;
  * This is a client side utility class that handles
  * common logic to to parameter reconfiguration.
  */
-public class ReconfigurationProtocolUtils {
+public final class ReconfigurationProtocolUtils {
   private ReconfigurationProtocolUtils() {
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303f80e6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java
new file mode 100644
index 0000000..6233024
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303f80e6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
index 29c7326..b2be9cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ReconfigurationProtocolServerSideUtils.java
@@ -32,7 +32,7 @@ import com.google.common.base.Optional;
  * This is a server side utility class that handles
  * common logic to to parameter reconfiguration.
  */
-public class ReconfigurationProtocolServerSideUtils {
+public final class ReconfigurationProtocolServerSideUtils {
   private ReconfigurationProtocolServerSideUtils() {
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303f80e6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 4b6e6e8..241f1e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -920,7 +920,7 @@ public class DataNode extends ReconfigurableBase
         .setSecretManager(blockPoolTokenSecretManager).build();
 
     ReconfigurationProtocolServerSideTranslatorPB reconfigurationProtocolXlator
-      = new ReconfigurationProtocolServerSideTranslatorPB(this);
+        = new ReconfigurationProtocolServerSideTranslatorPB(this);
     service = ReconfigurationProtocolService
         .newReflectiveBlockingService(reconfigurationProtocolXlator);
     DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class, service,


[36/37] hadoop git commit: HDFS-9514. TestDistributedFileSystem.testDFSClientPeerWriteTimeout failing; exception being swallowed. (Wei-Chiu Chuang via Yongjun Zhang)

Posted by as...@apache.org.
HDFS-9514. TestDistributedFileSystem.testDFSClientPeerWriteTimeout failing; exception being swallowed. (Wei-Chiu Chuang via Yongjun Zhang)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bf5295b1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bf5295b1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bf5295b1

Branch: refs/heads/yarn-2877
Commit: bf5295b118a24e762f21bb2b9054e5e0c1d1d14a
Parents: 2a4c7d4
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri Dec 11 19:52:38 2015 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Dec 11 19:52:38 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hadoop/hdfs/TestDistributedFileSystem.java  | 27 ++++++++++++--------
 2 files changed, 20 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf5295b1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 767bbcb..4c1287c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1745,6 +1745,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9519. Some coding improvement in SecondaryNameNode#main.
     (Xiao Chen via Yongjun Zhang)
 
+    HDFS-9514. TestDistributedFileSystem.testDFSClientPeerWriteTimeout failing;
+    exception being swallowed. (Wei-Chiu Chuang via Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf5295b1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 0543e9d..e628752 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -990,10 +990,14 @@ public class TestDistributedFileSystem {
         Assert.fail("read should timeout");
       } catch (SocketTimeoutException ste) {
         long delta = Time.now() - start;
-        Assert.assertTrue("read timedout too soon", delta >= timeout*0.9);
-        Assert.assertTrue("read timedout too late", delta <= timeout*1.1);
-      } catch (Throwable t) {
-        Assert.fail("wrong exception:"+t);
+        if (delta < timeout*0.9) {
+          throw new IOException("read timedout too soon in " + delta + " ms.",
+              ste);
+        }
+        if (delta > timeout*1.1) {
+          throw new IOException("read timedout too late in " + delta + " ms.",
+              ste);
+        }
       }
     } finally {
       cluster.shutdown();
@@ -1037,12 +1041,15 @@ public class TestDistributedFileSystem {
         Assert.fail("write finish in " + delta + " ms" + "but should timedout");
       } catch (SocketTimeoutException ste) {
         long delta = Time.now() - start;
-        Assert.assertTrue("write timedout too soon in " + delta + " ms",
-            delta >= timeout * 0.9);
-        Assert.assertTrue("write timedout too late in " + delta + " ms",
-            delta <= timeout * 1.2);
-      } catch (Throwable t) {
-        Assert.fail("wrong exception:" + t);
+
+        if (delta < timeout * 0.9) {
+          throw new IOException("write timedout too soon in " + delta + " ms.",
+              ste);
+        }
+        if (delta > timeout * 1.2) {
+          throw new IOException("write timedout too late in " + delta + " ms.",
+              ste);
+        }
       }
     } finally {
       cluster.shutdown();


[15/37] hadoop git commit: HADOOP-10729. Add tests for PB RPC in case version mismatch of client and server. Contributed by Junping Du.

Posted by as...@apache.org.
HADOOP-10729. Add tests for PB RPC in case version mismatch of client and server. Contributed by Junping Du.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c4084d9b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c4084d9b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c4084d9b

Branch: refs/heads/yarn-2877
Commit: c4084d9bc3b5c20405d9da6623b330d5720b64a1
Parents: 832b3cb
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Dec 9 05:41:44 2015 +0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Dec 9 05:41:44 2015 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../ipc/TestProtoBufRPCCompatibility.java       | 178 +++++++++++++++++++
 .../hadoop-common/src/test/proto/test.proto     |   8 +
 .../src/test/proto/test_rpc_service.proto       |  15 ++
 4 files changed, 204 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4084d9b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e7da77b..4cd295e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1497,6 +1497,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12609. Fix intermittent failure of TestDecayRpcScheduler.
     (Masatake Iwasaki via Arpit Agarwal)
 
+    HADOOP-10729. Add tests for PB RPC in case version mismatch of client and
+    server. (Junping Du via wheat9)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4084d9b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRPCCompatibility.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRPCCompatibility.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRPCCompatibility.java
new file mode 100644
index 0000000..76a93cf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRPCCompatibility.java
@@ -0,0 +1,178 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
+import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.OptRequestProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.OptResponseProto;
+
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.OldProtobufRpcProto;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.NewProtobufRpcProto;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.NewerProtobufRpcProto;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class TestProtoBufRPCCompatibility {
+
+  private static final String ADDRESS = "0.0.0.0";
+  public final static int PORT = 0;
+  private static InetSocketAddress addr;
+  private static RPC.Server server;
+  private static Configuration conf;
+
+  @ProtocolInfo(protocolName = "testProto", protocolVersion = 1)
+  public interface OldRpcService extends
+      OldProtobufRpcProto.BlockingInterface {
+  }
+
+  @ProtocolInfo(protocolName = "testProto", protocolVersion = 2)
+  public interface NewRpcService extends
+      NewProtobufRpcProto.BlockingInterface {
+  }
+
+  @ProtocolInfo(protocolName = "testProto", protocolVersion = 2)
+  public interface NewerRpcService extends
+      NewerProtobufRpcProto.BlockingInterface {
+  }
+
+  public static class OldServerImpl implements OldRpcService {
+
+    @Override
+    public EmptyResponseProto ping(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      // Ensure clientId is received
+      byte[] clientId = Server.getClientId();
+      Assert.assertNotNull(Server.getClientId());
+      Assert.assertEquals(16, clientId.length);
+      return EmptyResponseProto.newBuilder().build();
+    }
+
+    @Override
+    public EmptyResponseProto echo(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      // Ensure clientId is received
+      byte[] clientId = Server.getClientId();
+      Assert.assertNotNull(Server.getClientId());
+      Assert.assertEquals(16, clientId.length);
+      return EmptyResponseProto.newBuilder().build();
+    }
+  }
+
+  public static class NewServerImpl implements NewRpcService {
+
+    @Override
+    public EmptyResponseProto ping(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      // Ensure clientId is received
+      byte[] clientId = Server.getClientId();
+      Assert.assertNotNull(Server.getClientId());
+      Assert.assertEquals(16, clientId.length);
+      return EmptyResponseProto.newBuilder().build();
+    }
+
+    @Override
+    public OptResponseProto echo(RpcController unused, OptRequestProto request)
+        throws ServiceException {
+      return OptResponseProto.newBuilder().setMessage(request.getMessage())
+          .build();
+    }
+  }
+
+  @ProtocolInfo(protocolName = "testProto", protocolVersion = 2)
+  public static class NewerServerImpl implements NewerRpcService {
+
+    @Override
+    public EmptyResponseProto ping(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      // Ensure clientId is received
+      byte[] clientId = Server.getClientId();
+      Assert.assertNotNull(Server.getClientId());
+      Assert.assertEquals(16, clientId.length);
+      return EmptyResponseProto.newBuilder().build();
+    }
+
+    @Override
+    public EmptyResponseProto echo(RpcController unused, EmptyRequestProto request)
+        throws ServiceException {
+      // Ensure clientId is received
+      byte[] clientId = Server.getClientId();
+      Assert.assertNotNull(Server.getClientId());
+      Assert.assertEquals(16, clientId.length);
+      return EmptyResponseProto.newBuilder().build();
+    }
+  }
+
+  @Test
+  public void testProtocolVersionMismatch() throws IOException, ServiceException {
+    conf = new Configuration();
+    conf.setInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH, 1024);
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, NewRpcService.class, ProtobufRpcEngine.class);
+
+    // Create server side implementation
+    NewServerImpl serverImpl = new NewServerImpl();
+    BlockingService service = NewProtobufRpcProto
+        .newReflectiveBlockingService(serverImpl);
+    // Get RPC server for server side implementation
+    server = new RPC.Builder(conf).setProtocol(NewRpcService.class)
+        .setInstance(service).setBindAddress(ADDRESS).setPort(PORT).build();
+    addr = NetUtils.getConnectAddress(server);
+
+    server.start();
+
+    RPC.setProtocolEngine(conf, OldRpcService.class, ProtobufRpcEngine.class);
+
+    OldRpcService proxy = RPC.getProxy(OldRpcService.class, 0, addr, conf);
+    // Verify that exception is thrown if protocolVersion is mismatch between
+    // client and server.
+    EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
+    try {
+      proxy.ping(null, emptyRequest);
+      fail("Expected an exception to occur as version mismatch.");
+    } catch (Exception e) {
+      if (! (e.getMessage().contains("version mismatch"))){
+        // Exception type is not what we expected, re-throw it.
+        throw new IOException(e);
+      }
+    }
+
+    // Verify that missing of optional field is still compatible in RPC call.
+    RPC.setProtocolEngine(conf, NewerRpcService.class, ProtobufRpcEngine.class);
+    NewerRpcService newProxy = RPC.getProxy(NewerRpcService.class, 0, addr,
+        conf);
+    newProxy.echo(null, emptyRequest);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4084d9b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test.proto b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
index 9965f24..4ab590e 100644
--- a/hadoop-common-project/hadoop-common/src/test/proto/test.proto
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
@@ -35,6 +35,14 @@ message EchoResponseProto {
   required string message = 1;
 }
 
+message OptRequestProto {
+  optional string message = 1;
+}
+
+message OptResponseProto {
+  optional string message = 1;
+}
+
 message SleepRequestProto{
   required int32 milliSeconds = 1;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4084d9b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
index 4f64088..722af89 100644
--- a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
@@ -39,3 +39,18 @@ service TestProtobufRpc2Proto {
   rpc echo2(EchoRequestProto) returns (EchoResponseProto);
   rpc sleep(SleepRequestProto) returns (SleepResponseProto);
 }
+
+service OldProtobufRpcProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc echo(EmptyRequestProto) returns (EmptyResponseProto);
+}
+
+service NewProtobufRpcProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc echo(OptRequestProto) returns (OptResponseProto);
+}
+
+service NewerProtobufRpcProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc echo(EmptyRequestProto) returns (EmptyResponseProto);
+}


[12/37] hadoop git commit: HDFS-9273. Moving to 2.6.3 CHANGES section to reflect the backport.

Posted by as...@apache.org.
HDFS-9273. Moving to 2.6.3 CHANGES section to reflect the backport.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9676774e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9676774e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9676774e

Branch: refs/heads/yarn-2877
Commit: 9676774e233fa532c36e1ad18a41016d15be9f35
Parents: 303f80e
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Tue Dec 8 11:32:42 2015 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Dec 8 11:32:42 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9676774e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 169968f..d3e7e02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -2304,9 +2304,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-9274. Default value of dfs.datanode.directoryscan.throttle.limit.ms.per.sec
     should be consistent. (Yi Liu via zhz)
 
-    HDFS-9273. ACLs on root directory may be lost after NN restart.
-    (Xiao Chen via cnauroth)
-
     HDFS-9286. HttpFs does not parse ACL syntax correctly for operation
     REMOVEACLENTRIES. (Wei-Chiu Chuang via cnauroth)
 
@@ -3628,6 +3625,9 @@ Release 2.6.3 - UNRELEASED
     HDFS-9470. Encryption zone on root not loaded from fsimage after NN
     restart. (Xiao Chen via wang)
 
+    HDFS-9273. ACLs on root directory may be lost after NN restart.
+    (Xiao Chen via cnauroth)
+
 Release 2.6.2 - 2015-10-28
 
   INCOMPATIBLE CHANGES


[13/37] hadoop git commit: YARN-4248. Followup patch adding asf-licence exclusions for json test files

Posted by as...@apache.org.
YARN-4248. Followup patch adding asf-licence exclusions for json test files


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9f50e13d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9f50e13d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9f50e13d

Branch: refs/heads/yarn-2877
Commit: 9f50e13d5dc329c3a6df7f9bcaf2f29b35adc52b
Parents: 9676774
Author: Chris Douglas <cd...@apache.org>
Authored: Tue Dec 8 12:08:04 2015 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Tue Dec 8 12:08:04 2015 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-server-resourcemanager/pom.xml           | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f50e13d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 9d54184..a6972ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -321,6 +321,17 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/submit-reservation.json</exclude>
+            <exclude>src/test/resources/delete-reservation.json</exclude>
+            <exclude>src/test/resources/update-reservation.json</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 


[31/37] hadoop git commit: YARN-4341 addendum. Moved the changelog entry from 2.8.0 to 2.9.0.

Posted by as...@apache.org.
YARN-4341 addendum. Moved the changelog entry from 2.8.0 to 2.9.0.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b7b29236
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b7b29236
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b7b29236

Branch: refs/heads/yarn-2877
Commit: b7b292367ff611f5025605304db9f24a6cfe5653
Parents: e896458
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Dec 11 09:51:24 2015 -0800
Committer: Sangjin Lee <sj...@apache.org>
Committed: Fri Dec 11 09:51:24 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7b29236/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a29194a..2f6c66e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -67,6 +67,9 @@ Release 2.9.0 - UNRELEASED
     ResourceManager to support JvmPauseMonitor as a service.
     (Sunil G via Stevel)
 
+    YARN-4341. add doc about timeline performance tool usage (Chang Li via
+    sjlee)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -607,9 +610,6 @@ Release 2.8.0 - UNRELEASED
 
     YARN-4248. REST API for submit/update/delete Reservations. (curino)
 
-    YARN-4341. add doc about timeline performance tool usage (Chang Li via
-    sjlee)
-
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not


[05/37] hadoop git commit: YARN-4248. REST API for submit/update/delete Reservations. (curino)

Posted by as...@apache.org.
YARN-4248. REST API for submit/update/delete Reservations. (curino)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c25a6354
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c25a6354
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c25a6354

Branch: refs/heads/yarn-2877
Commit: c25a6354598ec855bec7f695a7c3eed8794cd381
Parents: 4ff973f
Author: = <cu...@apache.org>
Authored: Mon Dec 7 13:33:28 2015 -0800
Committer: = <cu...@apache.org>
Committed: Mon Dec 7 13:33:28 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../resourcemanager/webapp/RMWebServices.java   | 304 +++++++++++
 .../webapp/dao/ReservationDefinitionInfo.java   |  82 +++
 .../dao/ReservationDeleteRequestInfo.java       |  49 ++
 .../dao/ReservationDeleteResponseInfo.java      |  36 ++
 .../webapp/dao/ReservationRequestInfo.java      |  78 +++
 .../webapp/dao/ReservationRequestsInfo.java     |  63 +++
 .../dao/ReservationSubmissionRequestInfo.java   |  60 +++
 .../dao/ReservationSubmissionResponseInfo.java  |  54 ++
 .../dao/ReservationUpdateRequestInfo.java       |  60 +++
 .../dao/ReservationUpdateResponseInfo.java      |  37 ++
 .../webapp/TestRMWebServicesReservation.java    | 517 +++++++++++++++++++
 .../src/test/resources/delete-reservation.json  |   3 +
 .../src/test/resources/submit-reservation.json  |  31 ++
 .../src/test/resources/update-reservation.json  |  31 ++
 15 files changed, 1407 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 767e9c3..3ba94b4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -602,6 +602,8 @@ Release 2.8.0 - UNRELEASED
     YARN-3456. Improve handling of incomplete TimelineEntities. (Varun Saxena 
     via rohithsharmaks)
 
+    YARN-4248. REST API for submit/update/delete Reservations. (curino)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 2da477d..b744765 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -98,6 +98,11 @@ import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -145,6 +150,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntr
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDefinitionInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteResponseInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionResponseInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
@@ -1822,4 +1840,290 @@ public class RMWebServices {
     }
     return token;
   }
+
+  /**
+   * Function to submit a Reservation to the RM.
+   *
+   * @param resContext provides information to construct the
+   *          ReservationSubmissionRequest
+   * @param hsr the servlet request
+   * @return Response containing the status code
+   * @throws AuthorizationException
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @POST
+  @Path("/reservation/submit")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response submitReservation(
+      ReservationSubmissionRequestInfo resContext,
+      @Context HttpServletRequest hsr) throws AuthorizationException,
+      IOException, InterruptedException {
+
+    init();
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    if (callerUGI == null) {
+      throw new AuthorizationException("Unable to obtain user name, "
+          + "user not authenticated");
+    }
+    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
+      String msg = "The default static user cannot carry out this operation.";
+      return Response.status(Status.FORBIDDEN).entity(msg).build();
+    }
+
+    final ReservationSubmissionRequest reservation =
+        createReservationSubmissionRequest(resContext);
+
+    ReservationSubmissionResponseInfo resRespInfo;
+    try {
+      resRespInfo =
+          callerUGI.doAs(
+           new PrivilegedExceptionAction<ReservationSubmissionResponseInfo>() {
+                @Override
+                public ReservationSubmissionResponseInfo run()
+                    throws IOException, YarnException {
+                  ReservationSubmissionResponse tempRes =
+                      rm.getClientRMService().submitReservation(reservation);
+                  return new ReservationSubmissionResponseInfo(tempRes);
+                }
+              });
+    } catch (UndeclaredThrowableException ue) {
+      if (ue.getCause() instanceof YarnException) {
+        throw new BadRequestException(ue.getCause().getMessage());
+      }
+      LOG.info("Submit reservation request failed", ue);
+      throw ue;
+    }
+
+    return Response.status(Status.OK).entity(resRespInfo).build();
+  }
+
+  private ReservationSubmissionRequest createReservationSubmissionRequest(
+      ReservationSubmissionRequestInfo resContext) {
+
+    // defending against a couple of common submission format problems
+    if (resContext == null) {
+      throw new BadRequestException(
+          "Input ReservationSubmissionContext should not be null");
+    }
+    ReservationDefinitionInfo resInfo = resContext.getReservationDefinition();
+    if (resInfo == null) {
+      throw new BadRequestException(
+          "Input ReservationDefinition should not be null");
+    }
+
+    ReservationRequestsInfo resReqsInfo = resInfo.getReservationRequests();
+
+    if (resReqsInfo == null || resReqsInfo.getReservationRequest() == null
+        || resReqsInfo.getReservationRequest().size() == 0) {
+      throw new BadRequestException("The ReservationDefinition should"
+          + " contain at least one ReservationRequest");
+    }
+
+    ReservationRequestInterpreter[] values =
+        ReservationRequestInterpreter.values();
+    ReservationRequestInterpreter resInt =
+        values[resReqsInfo.getReservationRequestsInterpreter()];
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+
+    for (ReservationRequestInfo resReqInfo : resReqsInfo
+        .getReservationRequest()) {
+      ResourceInfo rInfo = resReqInfo.getCapability();
+      Resource capability =
+          Resource.newInstance(rInfo.getMemory(), rInfo.getvCores());
+      int numContainers = resReqInfo.getNumContainers();
+      int minConcurrency = resReqInfo.getMinConcurrency();
+      long duration = resReqInfo.getDuration();
+      ReservationRequest rr =
+          ReservationRequest.newInstance(capability, numContainers,
+              minConcurrency, duration);
+      list.add(rr);
+    }
+    ReservationRequests reqs = ReservationRequests.newInstance(list, resInt);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(resInfo.getArrival(),
+            resInfo.getDeadline(), reqs, resInfo.getReservationName());
+    ReservationSubmissionRequest request =
+        ReservationSubmissionRequest.newInstance(rDef, resContext.getQueue());
+
+    return request;
+  }
+
+  /**
+   * Function to update a Reservation to the RM.
+   *
+   * @param resContext provides information to construct the
+   *          ReservationUpdateRequest
+   * @param hsr the servlet request
+   * @return Response containing the status code
+   * @throws AuthorizationException
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @POST
+  @Path("/reservation/update")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      @Context HttpServletRequest hsr) throws AuthorizationException,
+      IOException, InterruptedException {
+
+    init();
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    if (callerUGI == null) {
+      throw new AuthorizationException("Unable to obtain user name, "
+          + "user not authenticated");
+    }
+    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
+      String msg = "The default static user cannot carry out this operation.";
+      return Response.status(Status.FORBIDDEN).entity(msg).build();
+    }
+
+    final ReservationUpdateRequest reservation =
+        createReservationUpdateRequest(resContext);
+
+    ReservationUpdateResponseInfo resRespInfo;
+    try {
+      resRespInfo =
+          callerUGI.doAs(
+              new PrivilegedExceptionAction<ReservationUpdateResponseInfo>() {
+                @Override
+                public ReservationUpdateResponseInfo run() throws IOException,
+                    YarnException {
+                  rm.getClientRMService().updateReservation(reservation);
+                  return new ReservationUpdateResponseInfo();
+                }
+              });
+    } catch (UndeclaredThrowableException ue) {
+      if (ue.getCause() instanceof YarnException) {
+        throw new BadRequestException(ue.getCause().getMessage());
+      }
+      LOG.info("Update reservation request failed", ue);
+      throw ue;
+    }
+
+    return Response.status(Status.OK).entity(resRespInfo).build();
+  }
+
+  private ReservationUpdateRequest createReservationUpdateRequest(
+      ReservationUpdateRequestInfo resContext) throws IOException {
+
+    // defending against a couple of common submission format problems
+    if (resContext == null) {
+      throw new BadRequestException(
+          "Input ReservationSubmissionContext should not be null");
+    }
+    ReservationDefinitionInfo resInfo = resContext.getReservationDefinition();
+    if (resInfo == null) {
+      throw new BadRequestException(
+          "Input ReservationDefinition should not be null");
+    }
+    ReservationRequestsInfo resReqsInfo = resInfo.getReservationRequests();
+    if (resReqsInfo == null || resReqsInfo.getReservationRequest() == null
+        || resReqsInfo.getReservationRequest().size() == 0) {
+      throw new BadRequestException("The ReservationDefinition should"
+          + " contain at least one ReservationRequest");
+    }
+    if (resContext.getReservationId() == null) {
+      throw new BadRequestException(
+          "Update operations must specify an existing ReservaitonId");
+    }
+
+    ReservationRequestInterpreter[] values =
+        ReservationRequestInterpreter.values();
+    ReservationRequestInterpreter resInt =
+        values[resReqsInfo.getReservationRequestsInterpreter()];
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+
+    for (ReservationRequestInfo resReqInfo : resReqsInfo
+        .getReservationRequest()) {
+      ResourceInfo rInfo = resReqInfo.getCapability();
+      Resource capability =
+          Resource.newInstance(rInfo.getMemory(), rInfo.getvCores());
+      int numContainers = resReqInfo.getNumContainers();
+      int minConcurrency = resReqInfo.getMinConcurrency();
+      long duration = resReqInfo.getDuration();
+      ReservationRequest rr =
+          ReservationRequest.newInstance(capability, numContainers,
+              minConcurrency, duration);
+      list.add(rr);
+    }
+    ReservationRequests reqs = ReservationRequests.newInstance(list, resInt);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(resInfo.getArrival(),
+            resInfo.getDeadline(), reqs, resInfo.getReservationName());
+    ReservationUpdateRequest request =
+        ReservationUpdateRequest.newInstance(rDef, ReservationId
+            .parseReservationId(resContext.getReservationId()));
+
+    return request;
+  }
+
+  /**
+   * Function to delete a Reservation to the RM.
+   *
+   * @param resContext provides information to construct
+   *          the ReservationDeleteRequest
+   * @param hsr the servlet request
+   * @return Response containing the status code
+   * @throws AuthorizationException
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @POST
+  @Path("/reservation/delete")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      @Context HttpServletRequest hsr) throws AuthorizationException,
+      IOException, InterruptedException {
+
+    init();
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    if (callerUGI == null) {
+      throw new AuthorizationException("Unable to obtain user name, "
+          + "user not authenticated");
+    }
+    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
+      String msg = "The default static user cannot carry out this operation.";
+      return Response.status(Status.FORBIDDEN).entity(msg).build();
+    }
+
+    final ReservationDeleteRequest reservation =
+        createReservationDeleteRequest(resContext);
+
+    ReservationDeleteResponseInfo resRespInfo;
+    try {
+      resRespInfo =
+          callerUGI.doAs(
+              new PrivilegedExceptionAction<ReservationDeleteResponseInfo>() {
+                @Override
+                public ReservationDeleteResponseInfo run() throws IOException,
+                    YarnException {
+                  rm.getClientRMService().deleteReservation(reservation);
+                  return new ReservationDeleteResponseInfo();
+                }
+              });
+    } catch (UndeclaredThrowableException ue) {
+      if (ue.getCause() instanceof YarnException) {
+        throw new BadRequestException(ue.getCause().getMessage());
+      }
+      LOG.info("Update reservation request failed", ue);
+      throw ue;
+    }
+
+    return Response.status(Status.OK).entity(resRespInfo).build();
+  }
+
+  private ReservationDeleteRequest createReservationDeleteRequest(
+      ReservationDeleteRequestInfo resContext) throws IOException {
+
+    ReservationDeleteRequest request =
+        ReservationDeleteRequest.newInstance(ReservationId
+            .parseReservationId(resContext.getReservationId()));
+
+    return request;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
new file mode 100644
index 0000000..ff82e75
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
@@ -0,0 +1,82 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class that represent a reservation definition.
+ */
+@XmlRootElement(name = "reservation-definition")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationDefinitionInfo {
+
+  @XmlElement(name = "arrival")
+  private long arrival;
+
+  @XmlElement(name = "deadline")
+  private long deadline;
+
+  @XmlElement(name = "reservation-requests")
+  private ReservationRequestsInfo reservationRequests;
+
+  @XmlElement(name = "reservation-name")
+  private String reservationName;
+
+  public ReservationDefinitionInfo() {
+
+  }
+
+  public long getArrival() {
+    return arrival;
+  }
+
+  public void setArrival(long arrival) {
+    this.arrival = arrival;
+  }
+
+  public long getDeadline() {
+    return deadline;
+  }
+
+  public void setDeadline(long deadline) {
+    this.deadline = deadline;
+  }
+
+  public ReservationRequestsInfo getReservationRequests() {
+    return reservationRequests;
+  }
+
+  public void setReservationRequests(
+      ReservationRequestsInfo reservationRequests) {
+    this.reservationRequests = reservationRequests;
+  }
+
+  public String getReservationName() {
+    return reservationName;
+  }
+
+  public void setReservationName(String reservationName) {
+    this.reservationName = reservationName;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteRequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteRequestInfo.java
new file mode 100644
index 0000000..2a6bde3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteRequestInfo.java
@@ -0,0 +1,49 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class represent the request of deleting a given reservation,
+ * selected by its id.
+ */
+@XmlRootElement(name = "reservation-delete-context")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationDeleteRequestInfo {
+
+  @XmlElement(name = "reservation-id")
+  private String reservationId;
+
+  public ReservationDeleteRequestInfo() {
+    reservationId = null;
+  }
+
+  public String getReservationId() {
+    return reservationId;
+  }
+
+  public void setReservationId(String reservationId) {
+    this.reservationId = reservationId;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteResponseInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteResponseInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteResponseInfo.java
new file mode 100644
index 0000000..51377a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDeleteResponseInfo.java
@@ -0,0 +1,36 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class that represent a reponse to a delete operation.
+ */
+@XmlRootElement(name = "reservation-delete-response")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationDeleteResponseInfo {
+
+  public ReservationDeleteResponseInfo() {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
new file mode 100644
index 0000000..ab82f3f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
@@ -0,0 +1,78 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class representing a reservation request.
+ */
+@XmlRootElement(name = "reservation-definition")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationRequestInfo {
+
+  @XmlElement(name = "capability")
+  private ResourceInfo capability;
+  @XmlElement(name = "min-concurrency")
+  private int minConcurrency;
+  @XmlElement(name = "num-containers")
+  private int numContainers;
+  @XmlElement(name = "duration")
+  private long duration;
+
+  public ReservationRequestInfo() {
+
+  }
+
+  public ResourceInfo getCapability() {
+    return capability;
+  }
+
+  public void setCapability(ResourceInfo capability) {
+    this.capability = capability;
+  }
+
+  public int getMinConcurrency() {
+    return minConcurrency;
+  }
+
+  public void setMinConcurrency(int minConcurrency) {
+    this.minConcurrency = minConcurrency;
+  }
+
+  public int getNumContainers() {
+    return numContainers;
+  }
+
+  public void setNumContainers(int numContainers) {
+    this.numContainers = numContainers;
+  }
+
+  public long getDuration() {
+    return duration;
+  }
+
+  public void setDuration(long duration) {
+    this.duration = duration;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
new file mode 100644
index 0000000..42cdb0e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class representing a list of ReservationRequest and the
+ * interpreter which capture the semantic of this list (all/any/order).
+ */
+@XmlRootElement(name = "reservation-definition")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationRequestsInfo {
+
+  @XmlElement(name = "reservation-request-interpreter")
+  private int reservationRequestsInterpreter;
+  @XmlElement(name = "reservation-request")
+  private ArrayList<ReservationRequestInfo> reservationRequest;
+
+  public ReservationRequestsInfo() {
+
+  }
+
+  public int getReservationRequestsInterpreter() {
+    return reservationRequestsInterpreter;
+  }
+
+  public void setReservationRequestsInterpreter(
+      int reservationRequestsInterpreter) {
+    this.reservationRequestsInterpreter = reservationRequestsInterpreter;
+  }
+
+  public ArrayList<ReservationRequestInfo> getReservationRequest() {
+    return reservationRequest;
+  }
+
+  public void setReservationRequest(
+      ArrayList<ReservationRequestInfo> reservationRequest) {
+    this.reservationRequest = reservationRequest;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java
new file mode 100644
index 0000000..701370d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java
@@ -0,0 +1,60 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class to allow users to send information required to create an
+ * ReservationSubmissionContext which can then be used to submit a reservation.
+ */
+@XmlRootElement(name = "reservation-submission-context")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationSubmissionRequestInfo {
+
+  @XmlElement(name = "queue")
+  private String queue;
+
+  @XmlElement(name = "reservation-definition")
+  private ReservationDefinitionInfo reservationDefinition;
+
+  public ReservationSubmissionRequestInfo() {
+  }
+
+  public String getQueue() {
+    return queue;
+  }
+
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  public ReservationDefinitionInfo getReservationDefinition() {
+    return reservationDefinition;
+  }
+
+  public void setReservationDefinition(
+      ReservationDefinitionInfo reservationDefinition) {
+    this.reservationDefinition = reservationDefinition;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java
new file mode 100644
index 0000000..943390b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java
@@ -0,0 +1,54 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+
+/**
+ * Simple class that represent a response to a reservation submission.
+ */
+@XmlRootElement(name = "reservation-submission-response")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationSubmissionResponseInfo {
+
+  @XmlElement(name = "reservation-id")
+  private String reservationId;
+
+  public ReservationSubmissionResponseInfo() {
+
+  }
+
+  public ReservationSubmissionResponseInfo(
+      ReservationSubmissionResponse response) {
+    this.reservationId = response.getReservationId().toString();
+  }
+
+  public String getReservationId() {
+    return reservationId;
+  }
+
+  public void setReservationId(String reservationId) {
+    this.reservationId = reservationId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateRequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateRequestInfo.java
new file mode 100644
index 0000000..cde24f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateRequestInfo.java
@@ -0,0 +1,60 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class to allow users to send information required to update an
+ * existing reservation.
+ */
+@XmlRootElement(name = "reservation-update-context")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationUpdateRequestInfo {
+
+  @XmlElement(name = "reservation-id")
+  private String reservationId;
+
+  @XmlElement(name = "reservation-definition")
+  private ReservationDefinitionInfo reservationDefinition;
+
+  public ReservationUpdateRequestInfo() {
+  }
+
+  public String getReservationId() {
+    return reservationId;
+  }
+
+  public void setReservationId(String reservationId) {
+    this.reservationId = reservationId;
+  }
+
+  public ReservationDefinitionInfo getReservationDefinition() {
+    return reservationDefinition;
+  }
+
+  public void setReservationDefinition(
+      ReservationDefinitionInfo reservationDefinition) {
+    this.reservationDefinition = reservationDefinition;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateResponseInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateResponseInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateResponseInfo.java
new file mode 100644
index 0000000..0a5cf59
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationUpdateResponseInfo.java
@@ -0,0 +1,37 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class that represent the response to a reservation update
+ * request.
+ */
+@XmlRootElement(name = "reservation-update-response")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationUpdateResponseInfo {
+
+  public ReservationUpdateResponseInfo() {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
new file mode 100644
index 0000000..bb0db81
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
@@ -0,0 +1,517 @@
+/**
+ * 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.webapp;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.Properties;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Singleton;
+import com.google.inject.servlet.GuiceServletContextListener;
+import com.google.inject.servlet.ServletModule;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.ClientResponse.Status;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONUnmarshaller;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.test.framework.WebAppDescriptor;
+
+@RunWith(Parameterized.class)
+public class TestRMWebServicesReservation extends JerseyTestBase {
+  private static MockRM rm;
+
+  private static Injector injector;
+  private String webserviceUserName = "testuser";
+
+  private boolean setAuthFilter = false;
+
+  private static final String TEST_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp")).getAbsolutePath();
+  private static final String FS_ALLOC_FILE = new File(TEST_DIR,
+      "test-fs-queues.xml").getAbsolutePath();
+
+  public static class GuiceServletConfig extends GuiceServletContextListener {
+
+    @Override
+    protected Injector getInjector() {
+      return injector;
+    }
+  }
+
+  /*
+   * Helper class to allow testing of RM web services which require
+   * authorization Add this class as a filter in the Guice injector for the
+   * MockRM
+   */
+
+  @Singleton
+  public static class TestRMCustomAuthFilter extends AuthenticationFilter {
+
+    @Override
+    protected Properties getConfiguration(String configPrefix,
+        FilterConfig filterConfig) throws ServletException {
+      Properties props = new Properties();
+      Enumeration<?> names = filterConfig.getInitParameterNames();
+      while (names.hasMoreElements()) {
+        String name = (String) names.nextElement();
+        if (name.startsWith(configPrefix)) {
+          String value = filterConfig.getInitParameter(name);
+          props.put(name.substring(configPrefix.length()), value);
+        }
+      }
+      props.put(AuthenticationFilter.AUTH_TYPE, "simple");
+      props.put(PseudoAuthenticationHandler.ANONYMOUS_ALLOWED, "false");
+      return props;
+    }
+
+  }
+
+  private abstract class TestServletModule extends ServletModule {
+    public Configuration conf = new Configuration();
+
+    public abstract void configureScheduler();
+
+    @Override
+    protected void configureServlets() {
+      configureScheduler();
+      bind(JAXBContextResolver.class);
+      bind(RMWebServices.class);
+      bind(GenericExceptionHandler.class);
+      conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+          YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
+      Configuration conf = new Configuration();
+      conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
+      conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+          YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
+      conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+          ResourceScheduler.class);
+      CapacitySchedulerConfiguration csconf =
+          new CapacitySchedulerConfiguration(conf);
+      String[] queues = { "default", "dedicated" };
+      csconf.setQueues("root", queues);
+      csconf.setCapacity("root.default", 50.0f);
+      csconf.setCapacity("root.dedicated", 50.0f);
+      csconf.setReservable("root.dedicated", true);
+
+      rm = new MockRM(csconf);
+      bind(ResourceManager.class).toInstance(rm);
+      if (setAuthFilter) {
+        filter("/*").through(TestRMCustomAuthFilter.class);
+      }
+      serve("/*").with(GuiceContainer.class);
+    }
+  }
+
+  private class CapTestServletModule extends TestServletModule {
+    @Override
+    public void configureScheduler() {
+      conf.set("yarn.resourcemanager.scheduler.class",
+          CapacityScheduler.class.getName());
+    }
+  }
+
+  private class FairTestServletModule extends TestServletModule {
+    @Override
+    public void configureScheduler() {
+      try {
+        PrintWriter out = new PrintWriter(new FileWriter(FS_ALLOC_FILE));
+        out.println("<?xml version=\"1.0\"?>");
+        out.println("<allocations>");
+        out.println("<queue name=\"root\">");
+        out.println("  <aclAdministerApps>someuser </aclAdministerApps>");
+        out.println("  <queue name=\"default\">");
+        out.println("    <aclAdministerApps>someuser </aclAdministerApps>");
+        out.println("  </queue>");
+        out.println("  <queue name=\"dedicated\">");
+        out.println("    <aclAdministerApps>someuser </aclAdministerApps>");
+        out.println("  </queue>");
+        out.println("</queue>");
+        out.println("</allocations>");
+        out.close();
+      } catch (IOException e) {
+      }
+      conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, FS_ALLOC_FILE);
+      conf.set("yarn.resourcemanager.scheduler.class",
+          FairScheduler.class.getName());
+    }
+  }
+
+  private Injector getNoAuthInjectorCap() {
+    return Guice.createInjector(new CapTestServletModule() {
+      @Override
+      protected void configureServlets() {
+        setAuthFilter = false;
+        super.configureServlets();
+      }
+    });
+  }
+
+  private Injector getSimpleAuthInjectorCap() {
+    return Guice.createInjector(new CapTestServletModule() {
+      @Override
+      protected void configureServlets() {
+        setAuthFilter = true;
+        conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+        // set the admin acls otherwise all users are considered admins
+        // and we can't test authorization
+        conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "testuser1");
+        super.configureServlets();
+      }
+    });
+  }
+
+  private Injector getNoAuthInjectorFair() {
+    return Guice.createInjector(new FairTestServletModule() {
+      @Override
+      protected void configureServlets() {
+        setAuthFilter = false;
+        super.configureServlets();
+      }
+    });
+  }
+
+  private Injector getSimpleAuthInjectorFair() {
+    return Guice.createInjector(new FairTestServletModule() {
+      @Override
+      protected void configureServlets() {
+        setAuthFilter = true;
+        conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+        // set the admin acls otherwise all users are considered admins
+        // and we can't test authorization
+        conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "testuser1");
+        super.configureServlets();
+      }
+    });
+  }
+
+  @Parameters
+  public static Collection<Object[]> guiceConfigs() {
+    return Arrays.asList(new Object[][] { { 0 }, { 1 }, { 2 }, { 3 } });
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  public TestRMWebServicesReservation(int run) {
+    super(new WebAppDescriptor.Builder(
+        "org.apache.hadoop.yarn.server.resourcemanager.webapp")
+        .contextListenerClass(GuiceServletConfig.class)
+        .filterClass(com.google.inject.servlet.GuiceFilter.class)
+        .clientConfig(new DefaultClientConfig(JAXBContextResolver.class))
+        .contextPath("jersey-guice-filter").servletPath("/").build());
+    switch (run) {
+    case 0:
+    default:
+      // No Auth Capacity Scheduler
+      injector = getNoAuthInjectorCap();
+      break;
+    case 1:
+      // Simple Auth Capacity Scheduler
+      injector = getSimpleAuthInjectorCap();
+      break;
+    case 2:
+      // No Auth Fair Scheduler
+      injector = getNoAuthInjectorFair();
+      break;
+    case 3:
+      // Simple Auth Fair Scheduler
+      injector = getSimpleAuthInjectorFair();
+      break;
+    }
+  }
+
+  private boolean isAuthenticationEnabled() {
+    return setAuthFilter;
+  }
+
+  private WebResource constructWebResource(WebResource r, String... paths) {
+    WebResource rt = r;
+    for (String path : paths) {
+      rt = rt.path(path);
+    }
+    if (isAuthenticationEnabled()) {
+      rt = rt.queryParam("user.name", webserviceUserName);
+    }
+    return rt;
+  }
+
+  private WebResource constructWebResource(String... paths) {
+    WebResource r = resource();
+    WebResource ws = r.path("ws").path("v1").path("cluster");
+    return this.constructWebResource(ws, paths);
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    if (rm != null) {
+      rm.stop();
+    }
+    super.tearDown();
+  }
+
+  @Test
+  public void testSubmitReservation() throws JSONException, Exception {
+    rm.start();
+    for (int i = 0; i < 100; i++) {
+      MockNM amNodeManager =
+          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
+      amNodeManager.nodeHeartbeat(true);
+    }
+    ReservationId rid =
+        testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON);
+    if (this.isAuthenticationEnabled()) {
+      assertNotNull(rid);
+    }
+    rm.stop();
+  }
+
+  @Test
+  public void testFailedSubmitReservation() throws JSONException, Exception {
+    rm.start();
+    // setup a cluster too small to accept the reservation
+    for (int i = 0; i < 1; i++) {
+      MockNM amNodeManager =
+          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
+      amNodeManager.nodeHeartbeat(true);
+    }
+    ReservationId rid =
+        testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON);
+    assertNull(rid);
+    rm.stop();
+  }
+
+  @Test
+  public void testUpdateReservation() throws JSONException, Exception {
+    rm.start();
+    for (int i = 0; i < 100; i++) {
+      MockNM amNodeManager =
+          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
+      amNodeManager.nodeHeartbeat(true);
+    }
+    ReservationId rid =
+        testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON);
+    if (this.isAuthenticationEnabled()) {
+      assertNotNull(rid);
+    }
+    testUpdateReservationHelper("reservation/update", rid,
+        MediaType.APPLICATION_JSON);
+
+    rm.stop();
+  }
+
+  @Test
+  public void testDeleteReservation() throws JSONException, Exception {
+    rm.start();
+    for (int i = 0; i < 100; i++) {
+      MockNM amNodeManager =
+          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
+      amNodeManager.nodeHeartbeat(true);
+    }
+    ReservationId rid =
+        testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON);
+    if (this.isAuthenticationEnabled()) {
+      assertNotNull(rid);
+    }
+    testDeleteReservationHelper("reservation/delete", rid,
+        MediaType.APPLICATION_JSON);
+
+    rm.stop();
+  }
+
+  private ReservationId testSubmissionReservationHelper(String path,
+      String media) throws JSONException, Exception {
+
+    String reservationJson = loadJsonFile("submit-reservation.json");
+
+    JSONJAXBContext jc =
+        new JSONJAXBContext(JSONConfiguration.mapped()
+            .build(), ReservationSubmissionRequestInfo.class);
+    JSONUnmarshaller unmarshaller = jc.createJSONUnmarshaller();
+    ReservationSubmissionRequestInfo rsci =
+        unmarshaller.unmarshalFromJSON(new StringReader(reservationJson),
+            ReservationSubmissionRequestInfo.class);
+
+    Thread.sleep(1000);
+    ClientResponse response =
+        constructWebResource(path).entity(rsci, MediaType.APPLICATION_JSON)
+            .accept(media).post(ClientResponse.class);
+
+    if (!this.isAuthenticationEnabled()) {
+      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      return null;
+    }
+
+    System.out.println("RESPONSE:" + response);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+
+    assertEquals("incorrect number of elements", 1, json.length());
+    ReservationId rid = null;
+    try {
+      rid = ReservationId.parseReservationId(json.getString("reservation-id"));
+      assertEquals("incorrect return value", rid.getId(), 1);
+    } catch (JSONException j) {
+      // failure is possible and is checked outside
+    }
+    return rid;
+  }
+
+  private void testUpdateReservationHelper(String path,
+      ReservationId reservationId, String media) throws JSONException,
+      Exception {
+
+    String reservationJson = loadJsonFile("update-reservation.json");
+
+    JSONJAXBContext jc =
+        new JSONJAXBContext(JSONConfiguration.mapped()
+            .build(), ReservationUpdateRequestInfo.class);
+    JSONUnmarshaller unmarshaller = jc.createJSONUnmarshaller();
+    ReservationUpdateRequestInfo rsci =
+        unmarshaller.unmarshalFromJSON(new StringReader(reservationJson),
+            ReservationUpdateRequestInfo.class);
+    if (this.isAuthenticationEnabled()) {
+      // only works when previous submit worked
+      if(rsci.getReservationId() == null) {
+        throw new IOException("Incorrectly parsed the reservatinId");
+      }
+      rsci.setReservationId(reservationId.toString());
+    }
+
+    Thread.sleep(1000);
+    ClientResponse response =
+        constructWebResource(path).entity(rsci, MediaType.APPLICATION_JSON)
+            .accept(media).post(ClientResponse.class);
+
+    if (!this.isAuthenticationEnabled()) {
+      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      return;
+    }
+
+    System.out.println("RESPONSE:" + response);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(Status.OK, response.getClientResponseStatus());
+
+  }
+
+  private String loadJsonFile(String filename) throws IOException {
+    ClassLoader cL = Thread.currentThread().getContextClassLoader();
+    if (cL == null) {
+      cL = Configuration.class.getClassLoader();
+    }
+    URL submitURI = cL.getResource(filename);
+
+    String reservationJson =
+        FileUtils.readFileToString(new File(submitURI.getFile()));
+    return reservationJson;
+  }
+
+  private void testDeleteReservationHelper(String path,
+      ReservationId reservationId, String media) throws JSONException,
+      Exception {
+
+    String reservationJson = loadJsonFile("delete-reservation.json");
+
+    JSONJAXBContext jc =
+        new JSONJAXBContext(JSONConfiguration.mapped()
+            .build(), ReservationDeleteRequestInfo.class);
+    JSONUnmarshaller unmarshaller = jc.createJSONUnmarshaller();
+    ReservationDeleteRequestInfo rsci =
+        unmarshaller.unmarshalFromJSON(new StringReader(reservationJson),
+            ReservationDeleteRequestInfo.class);
+    if (this.isAuthenticationEnabled()) {
+      // only works when previous submit worked
+      if(rsci.getReservationId() == null) {
+        throw new IOException("Incorrectly parsed the reservatinId");
+      }
+      rsci.setReservationId(reservationId.toString());
+    }
+
+    Thread.sleep(1000);
+    ClientResponse response =
+        constructWebResource(path).entity(rsci, MediaType.APPLICATION_JSON)
+            .accept(media).post(ClientResponse.class);
+
+    if (!this.isAuthenticationEnabled()) {
+      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      return;
+    }
+
+    System.out.println("RESPONSE:" + response);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(Status.OK, response.getClientResponseStatus());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/delete-reservation.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/delete-reservation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/delete-reservation.json
new file mode 100644
index 0000000..205d3d4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/delete-reservation.json
@@ -0,0 +1,3 @@
+{
+  "reservation-id" : "reservation_12341234_1"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
new file mode 100644
index 0000000..573d317
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
@@ -0,0 +1,31 @@
+{
+  "queue" : "dedicated",
+  "reservation-definition" : {
+     "arrival" : 1765541532000,
+     "deadline" : 1765542252000,
+     "reservation-name" : "res_1",
+     "reservation-requests" : {
+        "reservation-request-interpreter" : 0,
+        "reservation-request" : [
+           {
+             "duration" : 60,
+             "num-containers" : 220,
+             "min-concurrency" : 220,
+             "capability" : {
+               "memory" : 1024,
+               "vCores" : 1
+             }
+           },
+           {
+             "duration" : 120,
+             "num-containers" : 110,
+             "min-concurrency" : 110,
+             "capability" : {
+               "memory" : 1024,
+               "vCores" : 1
+             }
+           }
+        ]
+     }
+   }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c25a6354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/update-reservation.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/update-reservation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/update-reservation.json
new file mode 100644
index 0000000..df0df21
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/update-reservation.json
@@ -0,0 +1,31 @@
+{
+  "reservation-id" : "reservation_12341234_1",
+  "reservation-definition" : {
+     "arrival" : 1765541532000,
+     "deadline" : 1765542252000,
+     "reservation-name" : "res_1",
+     "reservation-requests" : {
+        "reservation-request-interpreter" : 0,
+        "reservation-request" : [
+           {
+             "duration" : 60,
+             "num-containers" : 100,
+             "min-concurrency" : 1,
+             "capability" : {
+               "memory" : 1024,
+               "vCores" : 1
+             }
+           },
+           {
+             "duration" : 120,
+             "num-containers" : 40,
+             "min-concurrency" : 1,
+             "capability" : {
+               "memory" : 1024,
+               "vCores" : 1
+             }
+           }
+          ]
+     }
+   }
+}
\ No newline at end of file


[25/37] hadoop git commit: YARN-3623-Addendum: Improve the description for Timeline Service Version configuration in yarn-default.xml

Posted by as...@apache.org.
YARN-3623-Addendum: Improve the description for Timeline Service Version configuration in yarn-default.xml


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/21daa6c6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/21daa6c6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/21daa6c6

Branch: refs/heads/yarn-2877
Commit: 21daa6c68a0bff51a14e748bf14d56b2f5a5580f
Parents: f910e4f
Author: Xuan <xg...@apache.org>
Authored: Thu Dec 10 10:56:12 2015 -0800
Committer: Xuan <xg...@apache.org>
Committed: Thu Dec 10 10:56:12 2015 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-common/src/main/resources/yarn-default.xml         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/21daa6c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 0917366..5b71b1f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1769,7 +1769,7 @@
     <description>Indicate what is the current version of the running
     timeline service. For example, if "yarn.timeline-service.version" is 1.5,
     and "yarn.timeline-service.enabled" is true, it means the cluster will and
-    should bring up the timeline service v.1.5 (and nothing else).
+    should bring up the timeline service v.1.5.
     On the client side, if the client uses the same version of timeline service,
     it should succeed. If the client chooses to use a smaller version in spite of this,
     then depending on how robust the compatibility story is between versions,


[02/37] hadoop git commit: YARN-3456. Improve handling of incomplete TimelineEntities. (Varun Saxena via rohithsharmaks)

Posted by as...@apache.org.
YARN-3456. Improve handling of incomplete TimelineEntities. (Varun Saxena via rohithsharmaks)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/01a641bc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/01a641bc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/01a641bc

Branch: refs/heads/yarn-2877
Commit: 01a641bc447c464b2830d58addd482f47dbd92ae
Parents: 65f3952
Author: rohithsharmaks <ro...@apache.org>
Authored: Mon Dec 7 15:16:56 2015 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Mon Dec 7 15:16:56 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../client/api/impl/TimelineClientImpl.java     |  8 ++++++--
 .../client/api/impl/TestTimelineClient.java     |  9 +++++++++
 .../server/timeline/TimelineDataManager.java    |  6 +++++-
 .../timeline/webapp/TimelineWebServices.java    |  2 ++
 .../webapp/TestTimelineWebServices.java         | 21 ++++++++++++++++++++
 6 files changed, 46 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01a641bc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 632c888..7c47126 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -599,6 +599,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4358. Reservation System: Improve relationship between SharingPolicy
     and ReservationAgent. (Carlo Curino via asuresh)
 
+    YARN-3456. Improve handling of incomplete TimelineEntities. (Varun Saxena 
+    via rohithsharmaks)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01a641bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index 04c84ca..019c7a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -29,7 +29,6 @@ import java.net.URL;
 import java.net.URLConnection;
 import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 
 import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.HttpsURLConnection;
@@ -301,7 +300,12 @@ public class TimelineClientImpl extends TimelineClient {
   public TimelinePutResponse putEntities(
       TimelineEntity... entities) throws IOException, YarnException {
     TimelineEntities entitiesContainer = new TimelineEntities();
-    entitiesContainer.addEntities(Arrays.asList(entities));
+    for (TimelineEntity entity : entities) {
+      if (entity.getEntityId() == null || entity.getEntityType() == null) {
+        throw new YarnException("Incomplete entity without entity id/type");
+      }
+      entitiesContainer.addEntity(entity);
+    }
     ClientResponse resp = doPosting(entitiesContainer, null);
     return resp.getEntity(TimelinePutResponse.class);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01a641bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
index 859a6c9..4c74c61 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
@@ -96,6 +96,15 @@ public class TestTimelineClient {
   }
 
   @Test
+  public void testPostIncompleteEntities() throws Exception {
+    try {
+      client.putEntities(new TimelineEntity());
+      Assert.fail("Exception should have been thrown");
+    } catch (YarnException e) {
+    }
+  }
+
+  @Test
   public void testPostEntitiesNoResponse() throws Exception {
     mockEntityClientResponse(
         client, ClientResponse.Status.INTERNAL_SERVER_ERROR, false, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01a641bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 459fd4e..23ff8e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -338,7 +339,10 @@ public class TimelineDataManager extends AbstractService {
           entity.getDomainId().length() == 0) {
         entity.setDomainId(DEFAULT_DOMAIN_ID);
       }
-
+      if (entity.getEntityId() == null || entity.getEntityType() == null) {
+        throw new BadRequestException("Incomplete entity without entity"
+            + " id/type");
+      }
       // check if there is existing entity
       TimelineEntity existingEntity = null;
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01a641bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
index 90f4d39..eb47ef2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
@@ -229,6 +229,8 @@ public class TimelineWebServices {
     }
     try {
       return timelineDataManager.postEntities(entities, callerUGI);
+    } catch (BadRequestException bre) {
+      throw bre;
     } catch (Exception e) {
       LOG.error("Error putting entities", e);
       throw new WebApplicationException(e,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01a641bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
index ab7cffd..74466a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
@@ -503,6 +503,27 @@ public class TestTimelineWebServices extends JerseyTestBase {
   }
 
   @Test
+  public void testPostIncompleteEntities() throws Exception {
+    TimelineEntities entities = new TimelineEntities();
+    TimelineEntity entity1 = new TimelineEntity();
+    entity1.setEntityId("test id 1");
+    entity1.setEntityType("test type 1");
+    entity1.setStartTime(System.currentTimeMillis());
+    entity1.setDomainId("domain_id_1");
+    entities.addEntity(entity1);
+    // Add an entity with no id or type.
+    entities.addEntity(new TimelineEntity());
+    WebResource r = resource();
+    // One of the entities has no id or type. HTTP 400 will be returned
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
+        .queryParam("user.name", "tester").accept(MediaType.APPLICATION_JSON)
+         .type(MediaType.APPLICATION_JSON).post(ClientResponse.class, entities);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(ClientResponse.Status.BAD_REQUEST,
+        response.getClientResponseStatus());
+  }
+
+  @Test
   public void testPostEntitiesWithYarnACLsEnabled() throws Exception {
     AdminACLsManager oldAdminACLsManager =
         timelineACLsManager.setAdminACLsManager(adminACLsManager);


[04/37] hadoop git commit: YARN-4422. Generic AHS sometimes doesn't show started, node, or logs on App page (Eric Payne via jeagles)

Posted by as...@apache.org.
YARN-4422. Generic AHS sometimes doesn't show started, node, or logs on App page (Eric Payne via jeagles)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4ff973f9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4ff973f9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4ff973f9

Branch: refs/heads/yarn-2877
Commit: 4ff973f96ae7f77cda3b52b38427e2991819ad31
Parents: 4546c75
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Mon Dec 7 15:04:48 2015 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Mon Dec 7 15:04:48 2015 -0600

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../ApplicationHistoryManagerOnTimelineStore.java         |  7 +++++++
 .../resourcemanager/metrics/AppAttemptFinishedEvent.java  | 10 +++++++++-
 .../resourcemanager/metrics/SystemMetricsPublisher.java   |  6 +++++-
 4 files changed, 24 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ff973f9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 360150a..767e9c3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1149,6 +1149,9 @@ Release 2.7.3 - UNRELEASED
     (with application having id > 9999) (Mohammad Shahid Khan & Varun Saxena
     via jianhe)
 
+    YARN-4422. Generic AHS sometimes doesn't show started, node, or logs on App page
+    (Eric Payne via jeagles)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ff973f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index 9b4e78e..6e6b9fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -488,6 +488,13 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
                     AppAttemptMetricsConstants.STATE_EVENT_INFO)
                     .toString());
           }
+          if (eventInfo
+              .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) {
+            amContainerId =
+                ConverterUtils.toContainerId(eventInfo.get(
+                    AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)
+                    .toString());
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ff973f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
index 71d9363..fc1d10f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.metrics;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 
@@ -31,6 +32,7 @@ public class AppAttemptFinishedEvent extends
   private String diagnosticsInfo;
   private FinalApplicationStatus appStatus;
   private YarnApplicationAttemptState state;
+  private ContainerId masterContainerId;
 
   public AppAttemptFinishedEvent(
       ApplicationAttemptId appAttemptId,
@@ -39,7 +41,8 @@ public class AppAttemptFinishedEvent extends
       String diagnosticsInfo,
       FinalApplicationStatus appStatus,
       YarnApplicationAttemptState state,
-      long finishedTime) {
+      long finishedTime,
+      ContainerId masterContainerId) {
     super(SystemMetricsEventType.APP_ATTEMPT_FINISHED, finishedTime);
     this.appAttemptId = appAttemptId;
     // This is the tracking URL after the application attempt is finished
@@ -48,6 +51,7 @@ public class AppAttemptFinishedEvent extends
     this.diagnosticsInfo = diagnosticsInfo;
     this.appStatus = appStatus;
     this.state = state;
+    this.masterContainerId = masterContainerId;
   }
 
   @Override
@@ -79,4 +83,8 @@ public class AppAttemptFinishedEvent extends
     return state;
   }
 
+  public ContainerId getMasterContainerId() {
+    return masterContainerId;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ff973f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
index 1bf7dbb..44cb807 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
@@ -186,7 +186,8 @@ public class SystemMetricsPublisher extends CompositeService {
               // based on app state if it doesn't exist
               app.getFinalApplicationStatus(),
               RMServerUtils.createApplicationAttemptState(appAttemtpState),
-              finishedTime));
+              finishedTime,
+              appAttempt.getMasterContainer().getId()));
     }
   }
 
@@ -416,6 +417,9 @@ public class SystemMetricsPublisher extends CompositeService {
         event.getFinalApplicationStatus().toString());
     eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO,
         event.getYarnApplicationAttemptState().toString());
+    eventInfo.put(
+        AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+        event.getMasterContainerId().toString());
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
     putEntity(entity);