Browse Source

Merging trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1454238 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
82230367ba
19 changed files with 189 additions and 44 deletions
  1. 26 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
  3. 6 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
  4. 7 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
  5. 21 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java
  6. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAZKUtil.java
  7. 9 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestProxyUserFromEnv.java
  8. 21 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java
  9. 23 2
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  11. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  12. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  13. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  14. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  15. 19 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  16. 6 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
  17. 16 1
      hadoop-mapreduce-project/CHANGES.txt
  18. 9 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobCounterInfo.java
  19. 13 1
      hadoop-yarn-project/CHANGES.txt

+ 26 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -456,7 +456,15 @@ Trunk (Unreleased)
     HADOOP-9372. Fix bad timeout annotations on tests.
     (Arpit Agarwal via suresh)
 
-Release 2.0.4-beta - UNRELEASED
+    HADOOP-9376. TestProxyUserFromEnv fails on a Windows domain joined machine.
+    (Ivan Mitic via suresh)
+
+    HADOOP-9365. TestHAZKUtil fails on Windows. (Ivan Mitic via suresh)
+
+    HADOOP-9364. PathData#expandAsGlob does not return correct results for
+    absolute paths on Windows. (Ivan Mitic via suresh)
+
+Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
@@ -527,6 +535,21 @@ Release 2.0.4-beta - UNRELEASED
     HADOOP-9337. org.apache.hadoop.fs.DF.getMount() does not work on Mac OS.
     (Ivan A. Veselovsky via atm)
 
+    HADOOP-9369. DNS#reverseDns() can return hostname with . appended at the
+    end. (Karthik Kambatla via atm)
+
+Release 2.0.4-alpha - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -1512,6 +1535,8 @@ Release 0.23.7 - UNRELEASED
     HADOOP-9209. Add shell command to dump file checksums (Todd Lipcon via
     jeagles)
 
+    HADOOP-9374. Add tokens from -tokenCacheFile into UGI (daryn)
+
   OPTIMIZATIONS
 
     HADOOP-8462. Native-code implementation of bzip2 codec. (Govind Kamat via

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java

@@ -338,7 +338,8 @@ public class PathData implements Comparable<PathData> {
       URI globUri = globPath.toUri();
       if (globUri.getScheme() != null) {
         globType = PathType.HAS_SCHEME;
-      } else if (new File(globUri.getPath()).isAbsolute()) {
+      } else if (!globUri.getPath().isEmpty() &&
+                 new Path(globUri.getPath()).isAbsolute()) {
         globType = PathType.SCHEMELESS_ABSOLUTE;
       } else {
         globType = PathType.RELATIVE;

+ 6 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java

@@ -89,7 +89,12 @@ public class DNS {
       ictx.close();
     }
 
-    return attribute.get("PTR").get().toString();
+    String hostname = attribute.get("PTR").get().toString();
+    int hostnameLength = hostname.length();
+    if (hostname.charAt(hostnameLength - 1) == '.') {
+      hostname = hostname.substring(0, hostnameLength - 1);
+    }
+    return hostname;
   }
 
   /**

+ 7 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java

@@ -42,6 +42,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * <code>GenericOptionsParser</code> is a utility to parse command line
@@ -321,15 +323,17 @@ public class GenericOptionsParser {
       String fileName = line.getOptionValue("tokenCacheFile");
       // check if the local file exists
       FileSystem localFs = FileSystem.getLocal(conf);
-      Path p = new Path(fileName);
+      Path p = localFs.makeQualified(new Path(fileName));
       if (!localFs.exists(p)) {
           throw new FileNotFoundException("File "+fileName+" does not exist.");
       }
       if(LOG.isDebugEnabled()) {
         LOG.debug("setting conf tokensFile: " + fileName);
       }
-      conf.set("mapreduce.job.credentials.json", localFs.makeQualified(p)
-          .toString(), "from -tokenCacheFile command line option");
+      UserGroupInformation.getCurrentUser().addCredentials(
+          Credentials.readTokenStorageFile(p, conf));
+      conf.set("mapreduce.job.credentials.json", p.toString(),
+               "from -tokenCacheFile command line option");
 
     }
   }

+ 21 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java

@@ -23,11 +23,13 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Shell;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -173,6 +175,25 @@ public class TestPathData {
         sortedString(testDir+"/d1/f1", testDir+"/d1/f1.1"),
         sortedString(items)
     );
+    
+    String absolutePathNoDriveLetter = testDir+"/d1/f1";
+    if (Shell.WINDOWS) {
+      // testDir is an absolute path with a drive letter on Windows, i.e.
+      // c:/some/path
+      // and for the test we want something like the following
+      // /some/path
+      absolutePathNoDriveLetter = absolutePathNoDriveLetter.substring(2);
+    }
+    items = PathData.expandAsGlob(absolutePathNoDriveLetter, conf);
+    assertEquals(
+        sortedString(absolutePathNoDriveLetter),
+        sortedString(items)
+    );
+    items = PathData.expandAsGlob(".", conf);
+    assertEquals(
+        sortedString("."),
+        sortedString(items)
+    );
   }
 
   @Test (timeout = 30000)

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

@@ -40,7 +40,8 @@ public class TestHAZKUtil {
       "test-file");
   
   /** A path which is expected not to exist */
-  private static final String BOGUS_FILE = "/xxxx-this-does-not-exist";
+  private static final String BOGUS_FILE =
+      new File("/xxxx-this-does-not-exist").getPath();
 
   @Test
   public void testEmptyACL() {

+ 9 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestProxyUserFromEnv.java

@@ -42,6 +42,15 @@ public class TestProxyUserFromEnv {
     BufferedReader br = new BufferedReader
                           (new InputStreamReader(pp.getInputStream()));
     String realUser = br.readLine().trim();
+
+    // On Windows domain joined machine, whoami returns the username
+    // in the DOMAIN\\username format, so we trim the domain part before
+    // the comparison. We don't have to special case for Windows
+    // given that Unix systems do not allow slashes in usernames.
+    int backslashIndex = realUser.indexOf('\\');
+    if (backslashIndex != -1) {
+      realUser = realUser.substring(backslashIndex + 1);
+    }
     assertEquals(realUser, realUgi.getUserName());
   }
 }

+ 21 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java

@@ -27,6 +27,11 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
@@ -164,13 +169,25 @@ public class TestGenericOptionsParser extends TestCase {
         th instanceof FileNotFoundException);
     
     // create file
-    Path tmpPath = new Path(tmpFile.toString());
-    localFs.create(tmpPath);
+    Path tmpPath = localFs.makeQualified(new Path(tmpFile.toString()));
+    Token<?> token = new Token<AbstractDelegationTokenIdentifier>(
+        "identifier".getBytes(), "password".getBytes(),
+        new Text("token-kind"), new Text("token-service"));
+    Credentials creds = new Credentials();
+    creds.addToken(new Text("token-alias"), token);
+    creds.writeTokenStorageFile(tmpPath, conf);
+
     new GenericOptionsParser(conf, args);
     String fileName = conf.get("mapreduce.job.credentials.json");
     assertNotNull("files is null", fileName);
-    assertEquals("files option does not match",
-      localFs.makeQualified(tmpPath).toString(), fileName);
+    assertEquals("files option does not match", tmpPath.toString(), fileName);
+    
+    Credentials ugiCreds =
+        UserGroupInformation.getCurrentUser().getCredentials();
+    assertEquals(1, ugiCreds.numberOfTokens());
+    Token<?> ugiToken = ugiCreds.getToken(new Text("token-alias"));
+    assertNotNull(ugiToken);
+    assertEquals(token, ugiToken);
     
     localFs.delete(new Path(testDir.getAbsolutePath()), true);
   }

+ 23 - 2
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -25,7 +25,6 @@ Trunk (Unreleased)
     HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
     Azure environments. (See breakdown of tasks below for subtasks and
     contributors)
-    
 
   IMPROVEMENTS
 
@@ -321,7 +320,7 @@ Trunk (Unreleased)
     HDFS-4297. Fix issues related to datanode concurrent reading and writing on
     Windows. (Arpit Agarwal, Chuan Liu via suresh)
 
-Release 2.0.4-beta - UNRELEASED
+Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
@@ -341,6 +340,9 @@ Release 2.0.4-beta - UNRELEASED
     HDFS-4519. Support overriding jsvc binary and log file locations
     when launching secure datanode. (Chris Nauroth via suresh)
 
+    HDFS-4569. Small image transfer related cleanups.
+    (Andrew Wang via suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -371,6 +373,21 @@ Release 2.0.4-beta - UNRELEASED
     HDFS-4544. Error in deleting blocks should not do check disk, for
     all types of errors. (Arpit Agarwal via suresh)
 
+    HDFS-4565. Use DFSUtil.getSpnegoKeytabKey() to get the spnego keytab key
+    in secondary namenode and namenode http server. (Arpit Gupta via suresh)
+
+Release 2.0.4-alpha - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.0.3-alpha - 2013-02-06
 
   INCOMPATIBLE CHANGES
@@ -2357,6 +2374,10 @@ Release 0.23.7 - UNRELEASED
     HDFS-4542. Webhdfs doesn't support secure proxy users (Daryn Sharp via
     kihwal)
 
+    HDFS-4560. Webhdfs cannot use tokens obtained by another user (daryn)
+
+    HDFS-4566. Webdhfs token cancelation should use authentication (daryn)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -361,7 +361,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   // Image transfer timeout
   public static final String DFS_IMAGE_TRANSFER_TIMEOUT_KEY = "dfs.image.transfer.timeout";
-  public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 60 * 1000;
+  public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 10 * 60 * 1000;
 
   //Keys with no defaults
   public static final String  DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";

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

@@ -113,11 +113,8 @@ public class NameNodeHttpServer {
                     DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY +
                     "' is not set.");
         }
-        String httpKeytab = conf.get(
-          DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
-        if (httpKeytab == null) {
-          httpKeytab = conf.get(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
-        }
+        String httpKeytab = conf.get(DFSUtil.getSpnegoKeytabKey(conf,
+            DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
         if (httpKeytab != null && !httpKeytab.isEmpty()) {
           params.put(
             DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,

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

@@ -255,15 +255,11 @@ public class SecondaryNameNode implements Runnable {
                                 new AccessControlList(conf.get(DFS_ADMIN, " "))) {
       {
         if (UserGroupInformation.isSecurityEnabled()) {
-          String httpKeytabKey = DFSConfigKeys.
-              DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
-          if (null == conf.get(httpKeytabKey)) {
-            httpKeytabKey = DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY;
-          }
           initSpnego(
               conf,
               DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
-              httpKeytabKey);
+              DFSUtil.getSpnegoKeytabKey(conf,
+                  DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY));
         }
       }
     };

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -229,7 +229,6 @@ public class TransferFsImage {
       SecurityUtil.openSecureHttpConnection(url);
 
     if (timeout <= 0) {
-      // Set the ping interval as timeout
       Configuration conf = new HdfsConfiguration();
       timeout = conf.getInt(DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_KEY,
           DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT);

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -341,7 +341,8 @@ public class WebHdfsFileSystem extends FileSystem
     boolean hasToken = false;
     if (UserGroupInformation.isSecurityEnabled() &&
         op != GetOpParam.Op.GETDELEGATIONTOKEN &&
-        op != PutOpParam.Op.RENEWDELEGATIONTOKEN) {
+        op != PutOpParam.Op.RENEWDELEGATIONTOKEN &&
+        op != PutOpParam.Op.CANCELDELEGATIONTOKEN) {
       synchronized (this) {
         hasToken = (delegationToken != null);
         if (hasToken) {
@@ -350,15 +351,15 @@ public class WebHdfsFileSystem extends FileSystem
         } // else we are talking to an insecure cluster
       }
     }
-    UserGroupInformation userUgi = ugi;
     if (!hasToken) {
+      UserGroupInformation userUgi = ugi;
       UserGroupInformation realUgi = userUgi.getRealUser();
       if (realUgi != null) { // proxy user
         authParams.add(new DoAsParam(userUgi.getShortUserName()));
         userUgi = realUgi;
       }
+      authParams.add(new UserParam(userUgi.getShortUserName()));
     }
-    authParams.add(new UserParam(userUgi.getShortUserName()));
     return authParams.toArray(new Param<?,?>[0]);
   }
 

+ 19 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -733,12 +733,29 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.image.transfer.timeout</name>
+  <value>600000</value>
+  <description>
+        Timeout for image transfer in milliseconds. This timeout and the related
+        dfs.image.transfer.bandwidthPerSec parameter should be configured such
+        that normal image transfer can complete within the timeout.
+        This timeout prevents client hangs when the sender fails during
+        image transfer, which is particularly important during checkpointing.
+        Note that this timeout applies to the entirety of image transfer, and
+        is not a socket timeout.
+  </description>
+</property>
+
 <property>
   <name>dfs.image.transfer.bandwidthPerSec</name>
   <value>0</value>
   <description>
-        Specifies the maximum amount of bandwidth that can be utilized for image
-        transfer in term of the number of bytes per second.
+        Maximum bandwidth used for image transfer in bytes per second.
+        This can help keep normal namenode operations responsive during
+        checkpointing. The maximum bandwidth and timeout in
+        dfs.image.transfer.timeout should be set such that normal image
+        transfers can complete successfully.
         A default value of 0 indicates that throttling is disabled. 
   </description>
 </property>

+ 6 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

@@ -134,7 +134,7 @@ public class TestWebHdfsUrl {
         },
         renewTokenUrl);
 
-    // send user+token
+    // send token
     URL cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
         fsPath, new TokenArgumentParam(tokenString));
     checkQueryParams(
@@ -142,16 +142,14 @@ public class TestWebHdfsUrl {
             PutOpParam.Op.CANCELDELEGATIONTOKEN.toQueryString(),
             new UserParam(ugi.getShortUserName()).toString(),
             new TokenArgumentParam(tokenString).toString(),
-            new DelegationParam(tokenString).toString()
         },
         cancelTokenUrl);
     
-    // send user+token
+    // send token
     URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString(),
             new DelegationParam(tokenString).toString()
         },
         fileStatusUrl);
@@ -219,24 +217,23 @@ public class TestWebHdfsUrl {
         },
         renewTokenUrl);
 
-    // send effective+token
+    // send token
     URL cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
         fsPath, new TokenArgumentParam(tokenString));
     checkQueryParams(
         new String[]{
             PutOpParam.Op.CANCELDELEGATIONTOKEN.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString(),
+            new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+            new DoAsParam(ugi.getShortUserName()).toString(),
             new TokenArgumentParam(tokenString).toString(),
-            new DelegationParam(tokenString).toString()
         },
         cancelTokenUrl);
     
-    // send effective+token
+    // send token
     URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
     checkQueryParams(
         new String[]{
             GetOpParam.Op.GETFILESTATUS.toQueryString(),
-            new UserParam(ugi.getShortUserName()).toString(),
             new DelegationParam(tokenString).toString()
         },
         fileStatusUrl);

+ 16 - 1
hadoop-mapreduce-project/CHANGES.txt

@@ -180,7 +180,7 @@ Trunk (Unreleased)
     HADOOP-9372. Fix bad timeout annotations on tests.
     (Arpit Agarwal via suresh)
 
-Release 2.0.4-beta - UNRELEASED
+Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
@@ -230,6 +230,18 @@ Release 2.0.4-beta - UNRELEASED
     appropriately used and that on-disk segments are correctly sorted on
     file-size. (Anty Rao and Ravi Prakash via acmurthy) 
 
+Release 2.0.4-alpha - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -792,6 +804,9 @@ Release 0.23.7 - UNRELEASED
     MAPREDUCE-5043. Fetch failure processing can cause AM event queue to
     backup and eventually OOM (Jason Lowe via bobby)
 
+    MAPREDUCE-5023. History Server Web Services missing Job Counters (Ravi
+    Prakash via tgraves)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -71,10 +71,15 @@ public class JobCounterInfo {
   }
 
   private void getCounters(AppContext ctx, Job job) {
-    total = new Counters();
     if (job == null) {
       return;
     }
+    total = job.getAllCounters();
+    boolean needTotalCounters = false;
+    if (total == null) {
+      total = new Counters();
+      needTotalCounters = true;
+    }
     map = new Counters();
     reduce = new Counters();
     // Get all types of counters
@@ -84,7 +89,6 @@ public class JobCounterInfo {
       if (counters == null) {
         continue;
       }
-      total.incrAllCounters(counters);
       switch (t.getType()) {
       case MAP:
         map.incrAllCounters(counters);
@@ -93,6 +97,9 @@ public class JobCounterInfo {
         reduce.incrAllCounters(counters);
         break;
       }
+      if (needTotalCounters) {
+        total.incrAllCounters(counters);
+      }
     }
   }
 

+ 13 - 1
hadoop-yarn-project/CHANGES.txt

@@ -48,7 +48,7 @@ Trunk - Unreleased
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
 
-Release 2.0.4-beta - UNRELEASED
+Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
@@ -91,6 +91,18 @@ Release 2.0.4-beta - UNRELEASED
     YARN-376. Fixes a bug which would prevent the NM knowing about completed
     containers and applications. (Jason Lowe via sseth)
 
+Release 2.0.4-alpha - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
     YARN-429. capacity-scheduler config missing from yarn-test artifact.
     (sseth via hitesh)