Browse Source

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1208002 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 years ago
parent
commit
ee97365f58

+ 7 - 7
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -6,9 +6,6 @@ Trunk (unreleased changes)
 
 
   NEW FEATURES
   NEW FEATURES
 
 
-    HADOOP-7777 Implement a base class for DNSToSwitchMapping implementations 
-    that can offer extra topology information. (stevel)
-
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm)
     HADOOP-7595. Upgrade dependency to Avro 1.5.3. (Alejandro Abdelnur via atm)
@@ -54,9 +51,6 @@ Trunk (unreleased changes)
     HADOOP-7728. Enable task memory management to be configurable in hadoop
     HADOOP-7728. Enable task memory management to be configurable in hadoop
     config setup script. (ramya)
     config setup script. (ramya)
 
 
-    HADOOP-7424. Log an error if the topology script doesn't handle multiple args.
-    (Uma Maheswara Rao G via eli)
-
     HADOOP-7792. Add verifyToken method to AbstractDelegationTokenSecretManager.
     HADOOP-7792. Add verifyToken method to AbstractDelegationTokenSecretManager.
     (jitendra)
     (jitendra)
 
 
@@ -114,7 +108,10 @@ Release 0.23.1 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
-  NEW FEATURES                                                                    
+  NEW FEATURES
+
+    HADOOP-7777 Implement a base class for DNSToSwitchMapping implementations 
+    that can offer extra topology information. (stevel)
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
@@ -126,6 +123,9 @@ Release 0.23.1 - Unreleased
     HADOOP-7858. Drop some info logging to DEBUG level in IPC,
     HADOOP-7858. Drop some info logging to DEBUG level in IPC,
     metrics, and HTTP. (todd via eli)
     metrics, and HTTP. (todd via eli)
 
 
+    HADOOP-7424. Log an error if the topology script doesn't handle multiple args.
+    (Uma Maheswara Rao G via eli)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 40 - 36
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -80,8 +80,6 @@ Trunk (unreleased changes)
                (Tomasz Nykiel via todd)
                (Tomasz Nykiel via todd)
 
 
   BUG FIXES
   BUG FIXES
-    HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
-
     HDFS-2299. TestOfflineEditsViewer is failing on trunk. (Uma Maheswara Rao G
     HDFS-2299. TestOfflineEditsViewer is failing on trunk. (Uma Maheswara Rao G
                via atm)
                via atm)
     HDFS-2310. TestBackupNode fails since HADOOP-7524 went in.
     HDFS-2310. TestBackupNode fails since HADOOP-7524 went in.
@@ -98,7 +96,7 @@ Trunk (unreleased changes)
     HDFS-46.   Change default namespace quota of root directory from
     HDFS-46.   Change default namespace quota of root directory from
     Integer.MAX_VALUE to Long.MAX_VALUE. (Uma Maheswara Rao G via szetszwo)
     Integer.MAX_VALUE to Long.MAX_VALUE. (Uma Maheswara Rao G via szetszwo)
 
 
-    HDFS-2373. Commands using webhdfs and hftp print unnecessary debug 
+    HDFS-2373. Commands using WebHDFS and hftp print unnecessary debug 
     info on the console with security enabled. (Arpit Gupta via suresh)
     info on the console with security enabled. (Arpit Gupta via suresh)
 
 
     HDFS-2349. Corruption detected during block transfers between DNs
     HDFS-2349. Corruption detected during block transfers between DNs
@@ -124,7 +122,7 @@ Release 0.23.1 - UNRELEASED
 
 
   NEW FEATURES
   NEW FEATURES
 
 
-    HDFS-2316. [umbrella] webhdfs: a complete FileSystem implementation for 
+    HDFS-2316. [umbrella] WebHDFS: a complete FileSystem implementation for 
     accessing HDFS over HTTP (szetszwo)
     accessing HDFS over HTTP (szetszwo)
 
 
   IMPROVEMENTS
   IMPROVEMENTS
@@ -190,6 +188,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2588. hdfs jsp pages missing DOCTYPE. (Dave Vronay via mattf)
     HDFS-2588. hdfs jsp pages missing DOCTYPE. (Dave Vronay via mattf)
 
 
     HDFS-2590. Fix the missing links in the WebHDFS forrest doc.  (szetszwo)
     HDFS-2590. Fix the missing links in the WebHDFS forrest doc.  (szetszwo)
+
+    HDFS-2596. TestDirectoryScanner doesn't test parallel scans. (eli)
     
     
 Release 0.23.0 - 2011-11-01 
 Release 0.23.0 - 2011-11-01 
 
 
@@ -500,24 +500,24 @@ Release 0.23.0 - 2011-11-01
     HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
     HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
     access to HDFS.  (szetszwo)
     access to HDFS.  (szetszwo)
 
 
-    HDFS-2317. Support read access to HDFS in webhdfs.  (szetszwo)
+    HDFS-2317. Support read access to HDFS in WebHDFS.  (szetszwo)
 
 
-    HDFS-2338. Add configuration option to enable/disable webhdfs.
+    HDFS-2338. Add configuration option to enable/disable WebHDFS.
     (jitendra via szetszwo)
     (jitendra via szetszwo)
 
 
-    HDFS-2318. Provide authentication to webhdfs using SPNEGO and delegation
+    HDFS-2318. Provide authentication to WebHDFS using SPNEGO and delegation
     tokens.  (szetszwo)
     tokens.  (szetszwo)
 
 
-    HDFS-2340. Support getFileBlockLocations and getDelegationToken in webhdfs.
+    HDFS-2340. Support getFileBlockLocations and getDelegationToken in WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-2348. Support getContentSummary and getFileChecksum in webhdfs.
+    HDFS-2348. Support getContentSummary and getFileChecksum in WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-2385. Support renew and cancel delegation tokens in webhdfs.
+    HDFS-2385. Support renew and cancel delegation tokens in WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-2539. Support doAs and GETHOMEDIRECTORY in webhdfs.
+    HDFS-2539. Support doAs and GETHOMEDIRECTORY in WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
   IMPROVEMENTS
   IMPROVEMENTS
@@ -915,19 +915,19 @@ Release 0.23.0 - 2011-11-01
     HDFS-2485. Improve code layout and constants in UnderReplicatedBlocks
     HDFS-2485. Improve code layout and constants in UnderReplicatedBlocks
     (stevel)
     (stevel)
 
 
-    HDFS-2356.  Support case insensitive query parameter names in webhdfs.
+    HDFS-2356.  Support case insensitive query parameter names in WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
     HDFS-2368.  Move SPNEGO conf properties from hdfs-default.xml to
     HDFS-2368.  Move SPNEGO conf properties from hdfs-default.xml to
     hdfs-site.xml.  (szetszwo)
     hdfs-site.xml.  (szetszwo)
 
 
-    HDFS-2395. Add a root element in the JSON responses of webhdfs.
+    HDFS-2395. Add a root element in the JSON responses of WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-2427. Change the default permission in webhdfs to 755 and add range
+    HDFS-2427. Change the default permission in WebHDFS to 755 and add range
     check/validation for all parameters.  (szetszwo)
     check/validation for all parameters.  (szetszwo)
 
 
-    HDFS-2501. Add version prefix and root methods to webhdfs.  (szetszwo)
+    HDFS-2501. Add version prefix and root methods to WebHDFS.  (szetszwo)
 
 
     HDFS-1869. mkdirs should use the supplied permission for all of the created
     HDFS-1869. mkdirs should use the supplied permission for all of the created
     directories. (Daryn Sharp via szetszwo)
     directories. (Daryn Sharp via szetszwo)
@@ -1310,30 +1310,30 @@ Release 0.23.0 - 2011-11-01
 
 
     HDFS-2409. _HOST in dfs.web.authentication.kerberos.principal. (jitendra)
     HDFS-2409. _HOST in dfs.web.authentication.kerberos.principal. (jitendra)
 
 
-    HDFS-2404. webhdfs liststatus json response is not correct. (suresh)
+    HDFS-2404. WebHDFS liststatus json response is not correct. (suresh)
 
 
     HDFS-2441. Remove the Content-Type set by HttpServer.QuotingInputFilter in
     HDFS-2441. Remove the Content-Type set by HttpServer.QuotingInputFilter in
-    webhdfs responses.  (szetszwo)
+    WebHDFS responses.  (szetszwo)
 
 
     HDFS-2428. Convert com.sun.jersey.api.ParamException$QueryParamException
     HDFS-2428. Convert com.sun.jersey.api.ParamException$QueryParamException
-    to IllegalArgumentException and response it as http BAD_REQUEST in webhdfs.
+    to IllegalArgumentException and response it as http BAD_REQUEST in WebHDFS.
     (szetszwo)
     (szetszwo)
 
 
     HDFS-2424. Added a root element "HdfsFileStatuses" for the response
     HDFS-2424. Added a root element "HdfsFileStatuses" for the response
-    of webhdfs listStatus.  (szetszwo)
+    of WebHDFS listStatus.  (szetszwo)
 
 
     MAPREDUCE-2764. Fix renewal of dfs delegation tokens. (Owen via jitendra)
     MAPREDUCE-2764. Fix renewal of dfs delegation tokens. (Owen via jitendra)
 
 
-    HDFS-2439. Fix NullPointerException in webhdfs when opening a non-existing
+    HDFS-2439. Fix NullPointerException in WebHDFS when opening a non-existing
     file or creating a file without specifying the replication parameter.
     file or creating a file without specifying the replication parameter.
     (szetszwo)
     (szetszwo)
 
 
-    HDFS-2453. Fix http response code for partial content in webhdfs, added
+    HDFS-2453. Fix http response code for partial content in WebHDFS, added
     getDefaultBlockSize() and getDefaultReplication() in WebHdfsFileSystem
     getDefaultBlockSize() and getDefaultReplication() in WebHdfsFileSystem
     and cleared content type in ExceptionHandler.  (szetszwo)
     and cleared content type in ExceptionHandler.  (szetszwo)
 
 
     HDFS-2411. The the auth to local mappings are not being respected, with 
     HDFS-2411. The the auth to local mappings are not being respected, with 
-    webhdfs enabled. (jitendra)
+    WebHDFS enabled. (jitendra)
 
 
     HDFS-2494. Close the streams and DFSClient in DatanodeWebHdfsMethods.
     HDFS-2494. Close the streams and DFSClient in DatanodeWebHdfsMethods.
     (Uma Maheswara Rao G via szetszwo)
     (Uma Maheswara Rao G via szetszwo)
@@ -1341,7 +1341,7 @@ Release 0.23.0 - 2011-11-01
     HDFS-2298. Fix TestDfsOverAvroRpc by changing ClientProtocol to
     HDFS-2298. Fix TestDfsOverAvroRpc by changing ClientProtocol to
     not include multiple methods of the same name. (cutting)
     not include multiple methods of the same name. (cutting)
 
 
-    HDFS-2432. Webhdfs: response FORBIDDEN when setReplication on non-files;
+    HDFS-2432. WebHDFS: response FORBIDDEN when setReplication on non-files;
     clear umask before creating a flie; throw IllegalArgumentException if
     clear umask before creating a flie; throw IllegalArgumentException if
     setOwner with both owner and group empty; throw FileNotFoundException if
     setOwner with both owner and group empty; throw FileNotFoundException if
     getFileStatus on non-existing files; fix bugs in getBlockLocations; and
     getFileStatus on non-existing files; fix bugs in getBlockLocations; and
@@ -1350,19 +1350,19 @@ Release 0.23.0 - 2011-11-01
     HDFS-2065. Add null checks in DFSClient.getFileChecksum(..).  (Uma
     HDFS-2065. Add null checks in DFSClient.getFileChecksum(..).  (Uma
     Maheswara Rao G via szetszwo)
     Maheswara Rao G via szetszwo)
 
 
-    HDFS-2416. distcp with a webhdfs uri on a secure cluster fails. (jitendra)
+    HDFS-2416. distcp with a WebHDFS uri on a secure cluster fails. (jitendra)
 
 
-    HDFS-2527. WebHdfs: remove the use of "Range" header in Open; use ugi
+    HDFS-2527. WebHDFS: remove the use of "Range" header in Open; use ugi
     username if renewer parameter is null in GetDelegationToken; response OK
     username if renewer parameter is null in GetDelegationToken; response OK
     when setting replication for non-files; rename GETFILEBLOCKLOCATIONS to
     when setting replication for non-files; rename GETFILEBLOCKLOCATIONS to
     GET_BLOCK_LOCATIONS and state that it is a private unstable API; replace
     GET_BLOCK_LOCATIONS and state that it is a private unstable API; replace
     isDirectory and isSymlink with enum {FILE, DIRECTORY, SYMLINK} in
     isDirectory and isSymlink with enum {FILE, DIRECTORY, SYMLINK} in
     HdfsFileStatus JSON object.  (szetszwo)
     HdfsFileStatus JSON object.  (szetszwo)
 
 
-    HDFS-2528. Webhdfs: set delegation kind to WEBHDFS and add a HDFS token
+    HDFS-2528. WebHDFS: set delegation kind to WEBHDFS and add a HDFS token
     when http requests are redirected to datanode.  (szetszwo)
     when http requests are redirected to datanode.  (szetszwo)
 
 
-    HDFS-2540. Webhdfs: change "Expect: 100-continue" to two-step write; change 
+    HDFS-2540. WebHDFS: change "Expect: 100-continue" to two-step write; change 
     "HdfsFileStatus" and "localName" respectively to "FileStatus" and
     "HdfsFileStatus" and "localName" respectively to "FileStatus" and
     "pathSuffix" in JSON response.  (szetszwo)
     "pathSuffix" in JSON response.  (szetszwo)
 
 
@@ -1964,10 +1964,21 @@ Release 0.22.0 - Unreleased
     HDFS-2002. Incorrect computation of needed blocks in getTurnOffTip().
     HDFS-2002. Incorrect computation of needed blocks in getTurnOffTip().
     (Plamen Jeliazkov via shv)
     (Plamen Jeliazkov via shv)
 
 
+    HDFS-2573. TestFiDataXceiverServer is failing, not testing OOME (cos)
+
     HDFS-2514. Link resolution bug for intermediate symlinks with
     HDFS-2514. Link resolution bug for intermediate symlinks with
     relative targets. (eli)
     relative targets. (eli)
 
 
-    HDFS-2573. TestFiDataXceiverServer is failing, not testing OOME (cos)
+    HDFS-1786. Some cli test cases expect a "null" message
+    (Uma Maheswara Rao G via todd)
+
+    HDFS-1855. TestDatanodeBlockScanner.testBlockCorruptionRecoveryPolicy()
+    part 2 fails in two different ways. (Matt Foley via eli)
+
+    HDFS-2346. TestHost2NodesMap & TestReplicasMap will fail depending upon
+    execution order of test methods (Laxman, Uma Maheswara Rao G via shv)
+
+    HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
 
 
 Release 0.21.1 - Unreleased
 Release 0.21.1 - Unreleased
 
 
@@ -2112,16 +2123,9 @@ Release 0.21.1 - Unreleased
     block placement and checkpoint/backup node features.  (Joe Crobak
     block placement and checkpoint/backup node features.  (Joe Crobak
     via szetszwo)
     via szetszwo)
 
 
-
     HDFS-1596. Replace fs.checkpoint.* with dfs.namenode.checkpoint.*
     HDFS-1596. Replace fs.checkpoint.* with dfs.namenode.checkpoint.*
     in documentations.  (Harsh J Chouraria via szetszwo)
     in documentations.  (Harsh J Chouraria via szetszwo)
 
 
-    HDFS-1786. Some cli test cases expect a "null" message
-    (Uma Maheswara Rao G via todd)
-
-    HDFS-1855. TestDatanodeBlockScanner.testBlockCorruptionRecoveryPolicy()
-    part 2 fails in two different ways. (Matt Foley via eli)
-
 Release 0.21.0 - 2010-08-13
 Release 0.21.0 - 2010-08-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -2590,7 +2594,7 @@ Release 0.21.0 - 2010-08-13
     HDFS-1012. hdfsproxy: Support for fully qualified HDFS path in addition to
     HDFS-1012. hdfsproxy: Support for fully qualified HDFS path in addition to
     simple unqualified path.  (Srikanth Sundarrajan via szetszwo)
     simple unqualified path.  (Srikanth Sundarrajan via szetszwo)
 
 
-    HDFS-933. Namenode should issue a delegation token only for kerberos 
+    HDFS-993. Namenode should issue a delegation token only for kerberos 
     authenticated clients.(jnp via boryas)
     authenticated clients.(jnp via boryas)
 
 
     HDFS-1087. Modify audit log to use a StringBuilder rather than a Formatter.
     HDFS-1087. Modify audit log to use a StringBuilder rather than a Formatter.

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelRead.java

@@ -156,8 +156,8 @@ public class TestParallelRead {
      */
      */
     private void pRead(int start, int len) throws Exception {
     private void pRead(int start, int len) throws Exception {
       assertTrue(
       assertTrue(
-          "Bad args: " + start + " + " + len + " should be < " + fileSize,
-          start + len < fileSize);
+          "Bad args: " + start + " + " + len + " should be <= " + fileSize,
+          start + len <= fileSize);
       DFSInputStream dis = testInfo.dis;
       DFSInputStream dis = testInfo.dis;
 
 
       byte buf[] = new byte[len];
       byte buf[] = new byte[len];

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -218,7 +218,7 @@ public class TestDirectoryScanner extends TestCase {
 
 
   public void testDirectoryScanner() throws Exception {
   public void testDirectoryScanner() throws Exception {
     // Run the test with and without parallel scanning
     // Run the test with and without parallel scanning
-    for (int parallelism = 1; parallelism < 2; parallelism++) {
+    for (int parallelism = 1; parallelism < 3; parallelism++) {
       runTest(parallelism);
       runTest(parallelism);
     }
     }
   }
   }

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

@@ -119,6 +119,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3413. RM web ui applications not sorted in any order by default.
     MAPREDUCE-3413. RM web ui applications not sorted in any order by default.
     (Jonathan Eagles via mahadev)
     (Jonathan Eagles via mahadev)
 
 
+    MAPREDUCE-3045. Fixed UI filters to not filter on hidden title-numeric
+    sort fields. (Jonathan Eagles via sseth)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -186,6 +189,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3265. Removed debug logs during job submission to LOG.debug to
     MAPREDUCE-3265. Removed debug logs during job submission to LOG.debug to
     cut down noise. (acmurthy) 
     cut down noise. (acmurthy) 
 
 
+    MAPREDUCE-3433. Finding counters by legacy group name returns empty
+    counters. (tomwhite)
+
 Release 0.23.0 - 2011-11-01 
 Release 0.23.0 - 2011-11-01 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 6 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/AbstractCounters.java

@@ -194,15 +194,16 @@ public abstract class AbstractCounters<C extends Counter,
    * @return the group
    * @return the group
    */
    */
   public synchronized G getGroup(String groupName) {
   public synchronized G getGroup(String groupName) {
-    boolean isFGroup = isFrameworkGroup(groupName);
-    G group = isFGroup ? fgroups.get(groupName) : groups.get(groupName);
+    String newGroupName = filterGroupName(groupName);
+    boolean isFGroup = isFrameworkGroup(newGroupName);
+    G group = isFGroup ? fgroups.get(newGroupName) : groups.get(newGroupName);
     if (group == null) {
     if (group == null) {
-      group = groupFactory.newGroup(filterGroupName(groupName), limits);
+      group = groupFactory.newGroup(newGroupName, limits);
       if (isFGroup) {
       if (isFGroup) {
-        fgroups.put(groupName, group);
+        fgroups.put(newGroupName, group);
       } else {
       } else {
         limits.checkGroups(groups.size() + 1);
         limits.checkGroups(groups.size() + 1);
-        groups.put(groupName, group);
+        groups.put(newGroupName, group);
       }
       }
     }
     }
     return group;
     return group;

+ 26 - 2
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestCounters.java → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java

@@ -17,16 +17,19 @@
  */
  */
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.text.ParseException;
 import java.text.ParseException;
 
 
+import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.mapreduce.TaskCounter;
+import org.junit.Test;
 
 
 /**
 /**
  * TestCounters checks the sanity and recoverability of {@code Counters}
  * TestCounters checks the sanity and recoverability of {@code Counters}
  */
  */
-public class TestCounters extends TestCase {
+public class TestCounters {
   enum myCounters {TEST1, TEST2};
   enum myCounters {TEST1, TEST2};
   private static final long MAX_VALUE = 10;
   private static final long MAX_VALUE = 10;
   
   
@@ -69,6 +72,7 @@ public class TestCounters extends TestCase {
                  counter.hashCode(), recoveredCounter.hashCode());
                  counter.hashCode(), recoveredCounter.hashCode());
   }
   }
   
   
+  @Test
   public void testCounters() throws IOException {
   public void testCounters() throws IOException {
     Enum[] keysWithResource = {TaskCounter.MAP_INPUT_RECORDS, 
     Enum[] keysWithResource = {TaskCounter.MAP_INPUT_RECORDS, 
                                TaskCounter.MAP_OUTPUT_BYTES};
                                TaskCounter.MAP_OUTPUT_BYTES};
@@ -92,6 +96,26 @@ public class TestCounters extends TestCase {
     }
     }
   }
   }
   
   
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testLegacyNames() {
+    Counters counters = new Counters();
+    counters.incrCounter(TaskCounter.MAP_INPUT_RECORDS, 1);
+    counters.incrCounter(JobCounter.DATA_LOCAL_MAPS, 1);
+    
+    assertEquals("New name", 1, counters.findCounter(
+        TaskCounter.class.getName(), "MAP_INPUT_RECORDS").getValue());
+    assertEquals("Legacy name", 1, counters.findCounter(
+        "org.apache.hadoop.mapred.Task$Counter",
+        "MAP_INPUT_RECORDS").getValue());
+
+    assertEquals("New name", 1, counters.findCounter(
+        JobCounter.class.getName(), "DATA_LOCAL_MAPS").getValue());
+    assertEquals("Legacy name", 1, counters.findCounter(
+        "org.apache.hadoop.mapred.JobInProgress$Counter",
+        "DATA_LOCAL_MAPS").getValue());
+  }
+  
   public static void main(String[] args) throws IOException {
   public static void main(String[] args) throws IOException {
     new TestCounters().testCounters();
     new TestCounters().testCounters();
   }
   }

+ 5 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/yarn.dt.plugins.js

@@ -2,6 +2,11 @@ if (!jQuery.fn.dataTableExt.fnVersionCheck("1.7.5")) {
   alert("These plugins requires dataTables 1.7.5+");
   alert("These plugins requires dataTables 1.7.5+");
 }
 }
 
 
+// don't filter on hidden html elements for an sType of title-numeric
+$.fn.dataTableExt.ofnSearch['title-numeric'] = function ( sData ) {
+   return sData.replace(/\n/g," ").replace( /<.*?>/g, "" );
+}
+
 // 'title-numeric' sort type
 // 'title-numeric' sort type
 jQuery.fn.dataTableExt.oSort['title-numeric-asc']  = function(a,b) {
 jQuery.fn.dataTableExt.oSort['title-numeric-asc']  = function(a,b) {
   var x = a.match(/title=["']?(-?\d+\.?\d*)/)[1];
   var x = a.match(/title=["']?(-?\d+\.?\d*)/)[1];

+ 1 - 1
hadoop-tools/hadoop-streaming/pom.xml

@@ -29,7 +29,7 @@
 
 
   <properties>
   <properties>
     <hadoop.log.dir>${project.build.directory}/log</hadoop.log.dir>
     <hadoop.log.dir>${project.build.directory}/log</hadoop.log.dir>
-    <test.exclude.pattern>%regex[.*(TestStreamingBadRecords|TestStreamingCombiner|TestStreamingStatus|TestUlimit).*]</test.exclude.pattern>
+    <test.exclude.pattern>%regex[.*(TestStreamingBadRecords|TestStreamingStatus|TestUlimit).*]</test.exclude.pattern>
   </properties>
   </properties>
 
 
   <dependencies>
   <dependencies>