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 vi...@apache.org on 2015/12/17 00:12:05 UTC

[1/3] hadoop git commit: Revert "MAPREDUCE-6566. Add retry support to mapreduce CLI tool. Contributed by Varun Vasudev"

Repository: hadoop
Updated Branches:
  refs/heads/trunk ce16541c6 -> 60fe6d53d


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60fe6d53/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 db87d9d..59f5ada 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,7 +26,6 @@ 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;
@@ -44,7 +43,6 @@ 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;
@@ -270,7 +268,7 @@ public class CLI extends Configured implements Tool {
         System.out.println("Created job " + job.getJobID());
         exitCode = 0;
       } else if (getStatus) {
-        Job job = getJob(JobID.forName(jobid));
+        Job job = cluster.getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -285,7 +283,7 @@ public class CLI extends Configured implements Tool {
           exitCode = 0;
         }
       } else if (getCounter) {
-        Job job = getJob(JobID.forName(jobid));
+        Job job = cluster.getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -301,7 +299,7 @@ public class CLI extends Configured implements Tool {
           }
         }
       } else if (killJob) {
-        Job job = getJob(JobID.forName(jobid));
+        Job job = cluster.getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -325,7 +323,7 @@ public class CLI extends Configured implements Tool {
           }
         }
       } else if (setJobPriority) {
-        Job job = getJob(JobID.forName(jobid));
+        Job job = cluster.getJob(JobID.forName(jobid));
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else {
@@ -341,7 +339,7 @@ public class CLI extends Configured implements Tool {
         viewHistory(historyFile, viewAllHistory);
         exitCode = 0;
       } else if (listEvents) {
-        listEvents(getJob(JobID.forName(jobid)), fromEvent, nEvents);
+        listEvents(cluster.getJob(JobID.forName(jobid)), fromEvent, nEvents);
         exitCode = 0;
       } else if (listJobs) {
         listJobs(cluster);
@@ -356,11 +354,11 @@ public class CLI extends Configured implements Tool {
         listBlacklistedTrackers(cluster);
         exitCode = 0;
       } else if (displayTasks) {
-        displayTasks(getJob(JobID.forName(jobid)), taskType, taskState);
+        displayTasks(cluster.getJob(JobID.forName(jobid)), taskType, taskState);
         exitCode = 0;
       } else if(killTask) {
         TaskAttemptID taskID = TaskAttemptID.forName(taskid);
-        Job job = getJob(taskID.getJobID());
+        Job job = cluster.getJob(taskID.getJobID());
         if (job == null) {
           System.out.println("Could not find job " + jobid);
         } else if (job.killTask(taskID, false)) {
@@ -372,7 +370,7 @@ public class CLI extends Configured implements Tool {
         }
       } else if(failTask) {
         TaskAttemptID taskID = TaskAttemptID.forName(taskid);
-        Job job = getJob(taskID.getJobID());
+        Job job = cluster.getJob(taskID.getJobID());
         if (job == null) {
             System.out.println("Could not find job " + jobid);
         } else if(job.killTask(taskID, true)) {
@@ -533,29 +531,6 @@ 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/60fe6d53/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 73f57d5..fdc916e 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,19 +20,14 @@ 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;
@@ -49,7 +44,7 @@ public class TestCLI {
     JobID jobId = JobID.forName(jobIdStr);
     Cluster mockCluster = mock(Cluster.class);
     Job job = mock(Job.class);
-    CLI cli = spy(new CLI(new Configuration()));
+    CLI cli = spy(new CLI());
 
     doReturn(mockCluster).when(cli).createCluster();
     when(job.getTaskReports(TaskType.MAP)).thenReturn(
@@ -117,7 +112,7 @@ public class TestCLI {
   @Test
   public void testJobKIll() throws Exception {
     Cluster mockCluster = mock(Cluster.class);
-    CLI cli = spy(new CLI(new Configuration()));
+    CLI cli = spy(new CLI());
     doReturn(mockCluster).when(cli).createCluster();
     String jobId1 = "job_1234654654_001";
     String jobId2 = "job_1234654654_002";
@@ -154,26 +149,4 @@ 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));
-    }
-  }
 }


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

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

This reverts commit fc470840a0b1f5dc8b3b13f7ed99fd68ba728216.


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

Branch: refs/heads/trunk
Commit: 60fe6d53d92c19c103219827fdb321780f1b33e5
Parents: ce16541
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Wed Dec 16 15:11:38 2015 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Wed Dec 16 15:11:38 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, 821 insertions(+), 646 deletions(-)
----------------------------------------------------------------------



[2/3] hadoop git commit: Revert "MAPREDUCE-6566. Add retry support to mapreduce CLI tool. Contributed by Varun Vasudev"

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/60fe6d53/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 499840b..1eb93e6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -1,5 +1,308 @@
 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
@@ -121,9 +424,6 @@ 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)
 
@@ -145,9 +445,6 @@ 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)
 
@@ -266,9 +563,6 @@ 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).
 
@@ -362,8 +656,6 @@ 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)
 
@@ -380,7 +672,6 @@ Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
-
   NEW FEATURES
 
   IMPROVEMENTS
@@ -456,8 +747,6 @@ 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
@@ -652,14 +941,17 @@ 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. [JDK8] Fix Javadoc errors caused by incorrect or illegal
-    tags in doc comments. (Akira AJISAKA and Andrew Purtell via ozawa)
+    MAPREDUCE-5657. 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)
 
@@ -791,8 +1083,6 @@ 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)
@@ -805,16 +1095,13 @@ 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 
@@ -863,24 +1150,27 @@ 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
@@ -888,9 +1178,6 @@ 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)
 
@@ -953,7 +1240,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)
@@ -961,7 +1248,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)
 
@@ -1131,7 +1418,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 
@@ -1214,9 +1501,6 @@ 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
@@ -1278,6 +1562,9 @@ 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)
 
@@ -2392,6 +2679,9 @@ 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)
 
@@ -2409,9 +2699,6 @@ 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)
 
@@ -2483,11 +2770,23 @@ Release 2.0.2-alpha - 2012-09-07
     MAPREDUCE-3921. MR AM should act on node health status changes. 
     (Bikas Saha via sseth)
 
-    MAPREDUCE-4355. Add RunningJob.getJobStatus() (kkambatl via tucu)
+    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-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)
 
@@ -2533,7 +2832,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)
 
@@ -2649,9 +2948,6 @@ 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)
 
@@ -2684,20 +2980,6 @@ 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
@@ -2754,6 +3036,9 @@ 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 
@@ -2948,7 +3233,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)
 
@@ -2995,7 +3280,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
@@ -3568,9 +3853,6 @@ 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
@@ -3582,7 +3864,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
@@ -3630,19 +3912,21 @@ 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)
 
@@ -3664,7 +3948,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
@@ -3674,9 +3958,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)
 
@@ -3756,39 +4040,20 @@ Release 0.23.2 - UNRELEASED
 
 Release 0.23.1 - 2012-02-17
 
-  NEW FEATURES
-
-    MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)
+  INCOMPATIBLE CHANGES
 
-    MAPREDUCE-3121. NodeManager should handle disk-failures (Ravi Gummadi via mahadev)
+  NEW FEATURES
 
-    MAPREDUCE-2863. Support web services for YARN and MR components. (Thomas
-    Graves via vinodkv)
+   MAPREDUCE-3121. NodeManager should handle disk-failures (Ravi Gummadi via mahadev)
 
-    MAPREDUCE-3251. Network ACLs can prevent some clients to talk to MR ApplicationMaster.
-    (Anupam Seth via mahadev)
+   MAPREDUCE-2863. Support web services for YARN and MR components. (Thomas
+   Graves via vinodkv)
 
-    MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)
+   MAPREDUCE-3251. Network ACLs can prevent some clients to talk to MR ApplicationMaster
+   (Anupam Seth via mahadev)
 
   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)
@@ -3802,26 +4067,23 @@ 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-3169. Create a new MiniMRCluster equivalent which only provides 
-    client APIs cross MR1 and MR2. (Ahmed via tucu)
+    MAPREDUCE-3415. improve MiniMRYarnCluster & DistributedShell JAR resolution.
+    (tucu)
 
     MAPREDUCE-3373. Hadoop scripts unconditionally source
     "$bin"/../libexec/hadoop-config.sh. (Bruno Mahé via tomwhite)
@@ -3829,7 +4091,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
@@ -3852,7 +4114,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
@@ -3883,10 +4145,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)
 
@@ -3904,12 +4166,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
@@ -3955,29 +4217,13 @@ 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)
 
@@ -3990,7 +4236,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
@@ -4002,8 +4248,6 @@ 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)
@@ -4014,7 +4258,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
@@ -4023,9 +4267,6 @@ 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)
@@ -4035,7 +4276,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
@@ -4044,10 +4285,7 @@ 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) 
-
-    MAPREDUCE-3468. Changed ant based infrastructure to use 0.23.1 version.
-    (sseth via acmurthy) 
+    cut down noise. (acmurthy)
 
     MAPREDUCE-3433. Finding counters by legacy group name returns empty
     counters. (tomwhite)
@@ -4055,12 +4293,9 @@ 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)
 
@@ -4078,27 +4313,22 @@ Release 0.23.1 - 2012-02-17
 
     MAPREDUCE-3479. JobClient#getJob cannot find local jobs. (tomwhite)
 
-    MAPREDUCE-3500. MRJobConfig creates an LD_LIBRARY_PATH using the platform ARCH. (tucu)
-
-    MAPREDUCE-3456. $HADOOP_PREFIX/bin/yarn should set defaults for 
+    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.
@@ -4110,10 +4340,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
@@ -4126,13 +4356,7 @@ Release 0.23.1 - 2012-02-17
     in the correct directory to work properly in secure mode. (Hitesh Shah via
     vinodkv)
 
-    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 
+    MAPREDUCE-3541. Fix broken TestJobQueueClient test. (Ravi Prakash via
     mahadev)
 
     MAPREDUCE-3398. Fixed log aggregation to work correctly in secure mode.
@@ -4147,7 +4371,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
@@ -4159,14 +4383,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
@@ -4181,27 +4405,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)
 
@@ -4209,25 +4433,45 @@ 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) 
+    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)
 
     MAPREDUCE-3624. Remove unnecessary dependency on JDK's tools.jar. (mahadev
     via acmurthy)
@@ -4238,7 +4482,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)
@@ -4246,8 +4490,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.
@@ -4265,13 +4509,10 @@ 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
@@ -4292,27 +4533,24 @@ Release 0.23.1 - 2012-02-17
     MAPREDUCE-3689. RM web UI doesn't handle newline in job name.
     (Thomas Graves via mahadev)
 
-    MAPREDUCE-3701. Delete HadoopYarnRPC from 0.23 branch.
-    (mahadev)
-
-    MAPREDUCE-3549. write api documentation for web service apis for RM, NM, 
+    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
@@ -4335,7 +4573,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)
@@ -4344,7 +4582,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)
@@ -4357,7 +4595,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)
@@ -4366,16 +4604,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)
@@ -4389,10 +4627,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
@@ -4411,14 +4649,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.
@@ -4428,14 +4666,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)
@@ -4444,21 +4682,24 @@ 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-901. (mahadev via acmurthy)
+
+    MAPREDUCE-3709. TestDistributedShell is failing. (Hitesh Shah via
+    mahadev)
 
     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
@@ -4467,9 +4708,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)
@@ -4478,7 +4719,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
@@ -4489,11 +4730,14 @@ 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-3843. Job summary log file found missing on the RM host 
+    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
     (Anupam Seth via tgraves)
 
     MAPREDUCE-3846. Addressed MR AM hanging issues during AM restart and then
@@ -4511,7 +4755,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
 
@@ -4525,12 +4769,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)
@@ -4543,14 +4787,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
@@ -4570,55 +4814,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)
@@ -4677,12 +4921,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.
@@ -4728,7 +4972,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.
@@ -4743,10 +4987,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.
@@ -4760,29 +5004,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).
@@ -4791,16 +5035,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()
@@ -4828,9 +5072,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)
@@ -4846,21 +5090,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)
 
@@ -4873,11 +5117,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
@@ -4885,7 +5129,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)
@@ -4895,12 +5139,6 @@ 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)
 
@@ -4908,7 +5146,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
 
@@ -4926,11 +5164,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.
@@ -4949,7 +5187,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.
@@ -4985,7 +5223,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
@@ -5005,7 +5243,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.
@@ -5059,7 +5297,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
@@ -5084,12 +5322,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)
@@ -5146,10 +5384,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)
@@ -5170,7 +5408,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)
@@ -5227,7 +5465,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)
 
@@ -5306,7 +5544,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.
@@ -5321,26 +5559,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)
@@ -5379,7 +5617,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)
 
@@ -5438,7 +5676,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)
@@ -5450,7 +5688,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)
 
@@ -5486,14 +5724,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)
 
@@ -5502,7 +5740,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)
@@ -5516,10 +5754,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)
@@ -5586,7 +5824,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)
@@ -5715,7 +5953,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)
@@ -5730,7 +5968,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)
 
@@ -5759,7 +5997,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)
@@ -5779,9 +6017,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)
@@ -5804,7 +6042,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.
@@ -5813,41 +6051,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) 
+    (Devaraj K via acmurthy)
+
+    MAPREDUCE-2879. Fix version for MR-279 to 0.23.0. (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
@@ -5855,17 +6093,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)
@@ -5873,42 +6111,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)
 
@@ -5928,18 +6166,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
@@ -5955,7 +6193,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)
@@ -5968,9 +6206,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)
@@ -5979,16 +6217,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)
 
@@ -6002,7 +6240,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)
@@ -6010,12 +6248,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)
@@ -6026,7 +6264,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
@@ -6041,7 +6279,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.
@@ -6065,16 +6303,13 @@ 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) 
-
-    MAPREDUCE-3137. Fix broken merge of MAPREDUCE-2179. (Hitesh Shah via
-    acmurthy) 
+    of the install properly. (Xie Xianshan 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)
@@ -6085,7 +6320,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
@@ -6101,19 +6336,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) 
+

<TRUNCATED>