浏览代码

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 年之前
父节点
当前提交
ee97365f58

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

@@ -6,9 +6,6 @@ Trunk (unreleased changes)
 
   NEW FEATURES
 
-    HADOOP-7777 Implement a base class for DNSToSwitchMapping implementations 
-    that can offer extra topology information. (stevel)
-
   IMPROVEMENTS
 
     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
     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.
     (jitendra)
 
@@ -114,7 +108,10 @@ Release 0.23.1 - Unreleased
 
   INCOMPATIBLE CHANGES
 
-  NEW FEATURES                                                                    
+  NEW FEATURES
+
+    HADOOP-7777 Implement a base class for DNSToSwitchMapping implementations 
+    that can offer extra topology information. (stevel)
 
   IMPROVEMENTS
 
@@ -126,6 +123,9 @@ Release 0.23.1 - Unreleased
     HADOOP-7858. Drop some info logging to DEBUG level in IPC,
     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
 
   BUG FIXES

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

@@ -80,8 +80,6 @@ Trunk (unreleased changes)
                (Tomasz Nykiel via todd)
 
   BUG FIXES
-    HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
-
     HDFS-2299. TestOfflineEditsViewer is failing on trunk. (Uma Maheswara Rao G
                via atm)
     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
     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)
 
     HDFS-2349. Corruption detected during block transfers between DNs
@@ -124,7 +122,7 @@ Release 0.23.1 - UNRELEASED
 
   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)
 
   IMPROVEMENTS
@@ -190,6 +188,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2588. hdfs jsp pages missing DOCTYPE. (Dave Vronay via mattf)
 
     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 
 
@@ -500,24 +500,24 @@ Release 0.23.0 - 2011-11-01
     HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
     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)
 
-    HDFS-2318. Provide authentication to webhdfs using SPNEGO and delegation
+    HDFS-2318. Provide authentication to WebHDFS using SPNEGO and delegation
     tokens.  (szetszwo)
 
-    HDFS-2340. Support getFileBlockLocations and getDelegationToken in webhdfs.
+    HDFS-2340. Support getFileBlockLocations and getDelegationToken in WebHDFS.
     (szetszwo)
 
-    HDFS-2348. Support getContentSummary and getFileChecksum in webhdfs.
+    HDFS-2348. Support getContentSummary and getFileChecksum in WebHDFS.
     (szetszwo)
 
-    HDFS-2385. Support renew and cancel delegation tokens in webhdfs.
+    HDFS-2385. Support renew and cancel delegation tokens in WebHDFS.
     (szetszwo)
 
-    HDFS-2539. Support doAs and GETHOMEDIRECTORY in webhdfs.
+    HDFS-2539. Support doAs and GETHOMEDIRECTORY in WebHDFS.
     (szetszwo)
 
   IMPROVEMENTS
@@ -915,19 +915,19 @@ Release 0.23.0 - 2011-11-01
     HDFS-2485. Improve code layout and constants in UnderReplicatedBlocks
     (stevel)
 
-    HDFS-2356.  Support case insensitive query parameter names in webhdfs.
+    HDFS-2356.  Support case insensitive query parameter names in WebHDFS.
     (szetszwo)
 
     HDFS-2368.  Move SPNEGO conf properties from hdfs-default.xml to
     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)
 
-    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)
 
-    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
     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-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
-    webhdfs responses.  (szetszwo)
+    WebHDFS responses.  (szetszwo)
 
     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)
 
     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)
 
-    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.
     (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
     and cleared content type in ExceptionHandler.  (szetszwo)
 
     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.
     (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
     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
     setOwner with both owner and group empty; throw FileNotFoundException if
     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
     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
     when setting replication for non-files; rename GETFILEBLOCKLOCATIONS to
     GET_BLOCK_LOCATIONS and state that it is a private unstable API; replace
     isDirectory and isSymlink with enum {FILE, DIRECTORY, SYMLINK} in
     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)
 
-    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
     "pathSuffix" in JSON response.  (szetszwo)
 
@@ -1964,10 +1964,21 @@ Release 0.22.0 - Unreleased
     HDFS-2002. Incorrect computation of needed blocks in getTurnOffTip().
     (Plamen Jeliazkov via shv)
 
+    HDFS-2573. TestFiDataXceiverServer is failing, not testing OOME (cos)
+
     HDFS-2514. Link resolution bug for intermediate symlinks with
     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
 
@@ -2112,16 +2123,9 @@ Release 0.21.1 - Unreleased
     block placement and checkpoint/backup node features.  (Joe Crobak
     via szetszwo)
 
-
     HDFS-1596. Replace fs.checkpoint.* with dfs.namenode.checkpoint.*
     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
 
   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
     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)
 
     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 {
       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;
 
       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 {
     // Run the test with and without parallel scanning
-    for (int parallelism = 1; parallelism < 2; parallelism++) {
+    for (int parallelism = 1; parallelism < 3; 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.
     (Jonathan Eagles via mahadev)
 
+    MAPREDUCE-3045. Fixed UI filters to not filter on hidden title-numeric
+    sort fields. (Jonathan Eagles via sseth)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -186,6 +189,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3265. Removed debug logs during job submission to LOG.debug to
     cut down noise. (acmurthy) 
 
+    MAPREDUCE-3433. Finding counters by legacy group name returns empty
+    counters. (tomwhite)
+
 Release 0.23.0 - 2011-11-01 
 
   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
    */
   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) {
-      group = groupFactory.newGroup(filterGroupName(groupName), limits);
+      group = groupFactory.newGroup(newGroupName, limits);
       if (isFGroup) {
-        fgroups.put(groupName, group);
+        fgroups.put(newGroupName, group);
       } else {
         limits.checkGroups(groups.size() + 1);
-        groups.put(groupName, group);
+        groups.put(newGroupName, 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;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
 import java.text.ParseException;
 
+import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.TaskCounter;
+import org.junit.Test;
 
 /**
  * TestCounters checks the sanity and recoverability of {@code Counters}
  */
-public class TestCounters extends TestCase {
+public class TestCounters {
   enum myCounters {TEST1, TEST2};
   private static final long MAX_VALUE = 10;
   
@@ -69,6 +72,7 @@ public class TestCounters extends TestCase {
                  counter.hashCode(), recoveredCounter.hashCode());
   }
   
+  @Test
   public void testCounters() throws IOException {
     Enum[] keysWithResource = {TaskCounter.MAP_INPUT_RECORDS, 
                                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 {
     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+");
 }
 
+// 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
 jQuery.fn.dataTableExt.oSort['title-numeric-asc']  = function(a,b) {
   var x = a.match(/title=["']?(-?\d+\.?\d*)/)[1];

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

@@ -29,7 +29,7 @@
 
   <properties>
     <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>
 
   <dependencies>