Переглянути джерело

Merge -r 493223:495045 from trunk, preparing for 0.10.1 release.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/branches/branch-0.10@495049 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 роки тому
батько
коміт
fe1584a519
43 змінених файлів з 3515 додано та 400 видалено
  1. 42 0
      CHANGES.txt
  2. 5 1
      bin/hadoop
  3. 12 1
      build.xml
  4. 32 23
      docs/index.html
  5. 22 22
      docs/index.pdf
  6. 9 0
      src/docs/src/documentation/content/xdocs/index.xml
  7. 8 1
      src/java/org/apache/hadoop/fs/FileUtil.java
  8. 1 1
      src/java/org/apache/hadoop/fs/s3/Jets3tFileSystemStore.java
  9. 27 13
      src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
  10. 114 0
      src/java/org/apache/hadoop/io/compress/BlockCompressorStream.java
  11. 129 0
      src/java/org/apache/hadoop/io/compress/BlockDecompressorStream.java
  12. 157 0
      src/java/org/apache/hadoop/io/compress/LzoCodec.java
  13. 319 0
      src/java/org/apache/hadoop/io/compress/lzo/LzoCompressor.java
  14. 308 0
      src/java/org/apache/hadoop/io/compress/lzo/LzoDecompressor.java
  15. 2 2
      src/java/org/apache/hadoop/mapred/InputFormatBase.java
  16. 55 30
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  17. 314 64
      src/java/org/apache/hadoop/mapred/JobTracker.java
  18. 8 10
      src/java/org/apache/hadoop/mapred/MapTask.java
  19. 61 14
      src/java/org/apache/hadoop/mapred/TaskInProgress.java
  20. 7 6
      src/java/org/apache/hadoop/mapred/TaskRunner.java
  21. 38 12
      src/java/org/apache/hadoop/mapred/TaskTracker.java
  22. 13 0
      src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
  23. 1 0
      src/java/org/apache/hadoop/util/NativeCodeLoader.java
  24. 11 7
      src/java/org/apache/hadoop/util/RunJar.java
  25. 1 1
      src/native/Makefile.am
  26. 1 1
      src/native/Makefile.in
  27. 2 0
      src/native/NEWS
  28. 36 0
      src/native/config.h.in
  29. 604 157
      src/native/configure
  30. 7 0
      src/native/configure.ac
  31. 253 0
      src/native/src/org/apache/hadoop/io/compress/lzo/LzoCompressor.c
  32. 194 0
      src/native/src/org/apache/hadoop/io/compress/lzo/LzoDecompressor.c
  33. 50 0
      src/native/src/org/apache/hadoop/io/compress/lzo/Makefile.am
  34. 469 0
      src/native/src/org/apache/hadoop/io/compress/lzo/Makefile.in
  35. 112 0
      src/native/src/org/apache/hadoop/io/compress/lzo/org_apache_hadoop_io_compress_lzo.h
  36. 5 5
      src/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c
  37. 5 5
      src/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c
  38. 0 6
      src/native/src/org/apache/hadoop/io/compress/zlib/org_apache_hadoop_io_compress_zlib.h
  39. 7 0
      src/native/src/org_apache_hadoop.h
  40. 47 14
      src/test/org/apache/hadoop/fs/s3/S3FileSystemBaseTest.java
  41. 20 3
      src/test/org/apache/hadoop/io/TestSequenceFile.java
  42. 6 0
      src/test/org/apache/hadoop/io/compress/TestCodec.java
  43. 1 1
      src/test/org/apache/hadoop/mapred/TestMapRed.java

+ 42 - 0
CHANGES.txt

@@ -1,6 +1,48 @@
 Hadoop Change Log
 
 
+Release 0.10.1 - 2007-01-10
+
+ 1. HADOOP-857.  Fix S3 FileSystem implementation to permit its use
+    for MapReduce input and output.  (Tom White via cutting)
+
+ 2. HADOOP-863.  Reduce logging verbosity introduced by HADOOP-813.
+    (Devaraj Das via cutting)
+
+ 3. HADOOP-815.  Fix memory leaks in JobTracker. (Arun C Murthy via cutting)
+
+ 4. HADOOP-600.  Fix a race condition in JobTracker.
+    (Arun C Murthy via cutting)
+
+ 5. HADOOP-864.  Fix 'bin/hadoop -jar' to operate correctly when
+    hadoop.tmp.dir does not yet exist.  (omalley via cutting)
+
+ 6. HADOOP-866.  Fix 'dfs -get' command to remove existing crc files,
+    if any.  (Milind Bhandarkar via cutting)
+
+ 7. HADOOP-871.  Fix a bug in bin/hadoop setting JAVA_LIBRARY_PATH.
+    (Arun C Murthy via cutting)
+
+ 8. HADOOP-868.  Decrease the number of open files during map,
+    respecting io.sort.factor.  (Devaraj Das via cutting)
+
+ 9. HADOOP-865.  Fix S3 FileSystem so that partially created files can
+    be deleted.  (Tom White via cutting)
+
+10. HADOOP-873.	 Pass java.library.path correctly to child processes.
+    (omalley via cutting)
+
+11. HADOOP-851.  Add support for the LZO codec.  This is much faster
+    than the default, zlib-based compression, but it is only available
+    when the native library is built.  (Arun C Murthy via cutting)
+
+12. HADOOP-880.  Fix S3 FileSystem to remove directories.
+    (Tom White via cutting)
+
+13. HADOOP-879.  Fix InputFormatBase to handle output generated by
+    MapFileOutputFormat.  (cutting)
+
+
 Release 0.10.0 - 2007-01-05
 
  1. HADOOP-763. Change DFS namenode benchmark to not use MapReduce.

+ 5 - 1
bin/hadoop

@@ -117,7 +117,11 @@ if [ -d "${HADOOP_HOME}/build/native" -o -d "${HADOOP_HOME}/lib/native" ]; then
   fi
   
   if [ -d "${HADOOP_HOME}/lib/native" ]; then
-    JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
+    if [ "x$JAVA_LIBRARY_PATH" != "x" ]; then
+      JAVA_LIBRARY_PATH=${JAVA_LIBRARY_PATH}:${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
+    else
+      JAVA_LIBRARY_PATH=${HADOOP_HOME}/lib/native/${JAVA_PLATFORM}
+    fi
   fi
 fi
  

+ 12 - 1
build.xml

@@ -9,7 +9,7 @@
  
   <property name="Name" value="Hadoop"/>
   <property name="name" value="hadoop"/>
-  <property name="version" value="0.10.1-dev"/>
+  <property name="version" value="0.10.2-dev"/>
   <property name="final.name" value="${name}-${version}"/>
   <property name="year" value="2006"/>
   <property name="libhdfs.version" value="1"/>
@@ -194,6 +194,7 @@
   	
     <mkdir dir="${build.native}/lib"/>
     <mkdir dir="${build.native}/src/org/apache/hadoop/io/compress/zlib"/>
+    <mkdir dir="${build.native}/src/org/apache/hadoop/io/compress/lzo"/>
 
   	<javah
   	  classpath="${build.classes}"
@@ -205,6 +206,16 @@
       <class name="org.apache.hadoop.io.compress.zlib.ZlibDecompressor" />
   	</javah>
 
+  	<javah
+  	  classpath="${build.classes}"
+  	  destdir="${build.native}/src/org/apache/hadoop/io/compress/lzo"
+      force="yes"
+  	  verbose="yes"
+  	  >
+  	  <class name="org.apache.hadoop.io.compress.lzo.LzoCompressor" />
+      <class name="org.apache.hadoop.io.compress.lzo.LzoDecompressor" />
+  	</javah>
+
 	<exec dir="${build.native}" executable="sh" failonerror="true">
 	  <env key="OS_NAME" value="${os.name}"/>
 	  <env key="OS_ARCH" value="${os.arch}"/>

+ 32 - 23
docs/index.html

@@ -128,6 +128,9 @@ document.write("<text>Last Published:</text> " + document.lastModified);
 <a href="#News">News</a>
 <ul class="minitoc">
 <li>
+<a href="#10+January%2C+2007%3A+release+0.10.1+available">10 January, 2007: release 0.10.1 available</a>
+</li>
+<li>
 <a href="#5+January%2C+2007%3A+release+0.10.0+available">5 January, 2007: release 0.10.0 available</a>
 </li>
 <li>
@@ -205,115 +208,121 @@ document.write("<text>Last Published:</text> " + document.lastModified);
 <a name="N1000C"></a><a name="News"></a>
 <h2 class="h3">News</h2>
 <div class="section">
-<a name="N10012"></a><a name="5+January%2C+2007%3A+release+0.10.0+available"></a>
+<a name="N10012"></a><a name="10+January%2C+2007%3A+release+0.10.1+available"></a>
+<h3 class="h4">10 January, 2007: release 0.10.1 available</h3>
+<p>This fixes critical bugs in 0.10.0.  For details see the <a href="http://tinyurl.com/ycpsms">release notes</a>. The release can
+      be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
+      nearby mirror</a>.
+      </p>
+<a name="N10024"></a><a name="5+January%2C+2007%3A+release+0.10.0+available"></a>
 <h3 class="h4">5 January, 2007: release 0.10.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/yapkxs">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10024"></a><a name="15+December%2C+2006%3A+release+0.9.2+available"></a>
+<a name="N10036"></a><a name="15+December%2C+2006%3A+release+0.9.2+available"></a>
 <h3 class="h4">15 December, 2006: release 0.9.2 available</h3>
 <p>This fixes critical bugs in 0.9.1.  For details see the <a href="http://tinyurl.com/ya8lfd">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10036"></a><a name="6+December%2C+2006%3A+release+0.9.1+available"></a>
+<a name="N10048"></a><a name="6+December%2C+2006%3A+release+0.9.1+available"></a>
 <h3 class="h4">6 December, 2006: release 0.9.1 available</h3>
 <p>This fixes critical bugs in 0.9.0.  For details see the <a href="http://tinyurl.com/y55d7p">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10048"></a><a name="1+December%2C+2006%3A+release+0.9.0+available"></a>
+<a name="N1005A"></a><a name="1+December%2C+2006%3A+release+0.9.0+available"></a>
 <h3 class="h4">1 December, 2006: release 0.9.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/sdjhb">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N1005A"></a><a name="3+November%2C+2006%3A+release+0.8.0+available"></a>
+<a name="N1006C"></a><a name="3+November%2C+2006%3A+release+0.8.0+available"></a>
 <h3 class="h4">3 November, 2006: release 0.8.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/ykqv6s">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N1006C"></a><a name="18+October%2C+2006%3A+release+0.7.2+available"></a>
+<a name="N1007E"></a><a name="18+October%2C+2006%3A+release+0.7.2+available"></a>
 <h3 class="h4">18 October, 2006: release 0.7.2 available</h3>
 <p>This fixes critical bugs in 0.7.1.  For details see the <a href="http://tinyurl.com/ygy6y7">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N1007E"></a><a name="11+October%2C+2006%3A+release+0.7.1+available"></a>
+<a name="N10090"></a><a name="11+October%2C+2006%3A+release+0.7.1+available"></a>
 <h3 class="h4">11 October, 2006: release 0.7.1 available</h3>
 <p>This fixes critical bugs in 0.7.0.  For details see the <a href="http://tinyurl.com/p7qod">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10090"></a><a name="6+October%2C+2006%3A+release+0.7.0+available"></a>
+<a name="N100A2"></a><a name="6+October%2C+2006%3A+release+0.7.0+available"></a>
 <h3 class="h4">6 October, 2006: release 0.7.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/kvd9m">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N100A2"></a><a name="18+September%2C+2006%3A+release+0.6.2+available"></a>
+<a name="N100B4"></a><a name="18+September%2C+2006%3A+release+0.6.2+available"></a>
 <h3 class="h4">18 September, 2006: release 0.6.2 available</h3>
 <p>This fixes critical bugs in 0.6.1.  For details see the <a href="http://tinyurl.com/gyb56">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N100B4"></a><a name="13+September%2C+2006%3A+release+0.6.1+available"></a>
+<a name="N100C6"></a><a name="13+September%2C+2006%3A+release+0.6.1+available"></a>
 <h3 class="h4">13 September, 2006: release 0.6.1 available</h3>
 <p>This fixes critical bugs in 0.6.0.  For details see the <a href="http://tinyurl.com/lykp4">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N100C6"></a><a name="8+September%2C+2006%3A+release+0.6.0+available"></a>
+<a name="N100D8"></a><a name="8+September%2C+2006%3A+release+0.6.0+available"></a>
 <h3 class="h4">8 September, 2006: release 0.6.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/r3zoj">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N100D8"></a><a name="4+August%2C+2006%3A+release+0.5.0+available"></a>
+<a name="N100EA"></a><a name="4+August%2C+2006%3A+release+0.5.0+available"></a>
 <h3 class="h4">4 August, 2006: release 0.5.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/pnml2">release notes</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N100EA"></a><a name="28+June%2C+2006%3A+release+0.4.0+available"></a>
+<a name="N100FC"></a><a name="28+June%2C+2006%3A+release+0.4.0+available"></a>
 <h3 class="h4">28 June, 2006: release 0.4.0 available</h3>
 <p>For details see the <a href="http://tinyurl.com/o35b6">change log</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N100FC"></a><a name="9+June%2C+2006%3A+release+0.3.2+available"></a>
+<a name="N1010E"></a><a name="9+June%2C+2006%3A+release+0.3.2+available"></a>
 <h3 class="h4">9 June, 2006: release 0.3.2 available</h3>
 <p>This is a bugfix release.  For details see the <a href="http://tinyurl.com/k9g5c">change log</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N1010E"></a><a name="8+June%2C+2006%3A+FAQ+added+to+Wiki"></a>
+<a name="N10120"></a><a name="8+June%2C+2006%3A+FAQ+added+to+Wiki"></a>
 <h3 class="h4">8 June, 2006: FAQ added to Wiki</h3>
 <p>Hadoop now has a <a href="http://wiki.apache.org/lucene-hadoop/FAQ">FAQ</a>.  Please
       help make this more complete!
       </p>
-<a name="N1011C"></a><a name="5+June%2C+2006%3A+release+0.3.1+available"></a>
+<a name="N1012E"></a><a name="5+June%2C+2006%3A+release+0.3.1+available"></a>
 <h3 class="h4">5 June, 2006: release 0.3.1 available</h3>
 <p>This is a bugfix release.  For details see the <a href="http://tinyurl.com/l6on4">change log</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N1012E"></a><a name="2+June%2C+2006%3A+release+0.3.0+available"></a>
+<a name="N10140"></a><a name="2+June%2C+2006%3A+release+0.3.0+available"></a>
 <h3 class="h4">2 June, 2006: release 0.3.0 available</h3>
 <p>This includes many fixes, improving performance, scalability
       and reliability and adding new features.  For details see the <a href="http://tinyurl.com/rq3f7">change log</a>. The release can
       be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10140"></a><a name="12+May%2C+2006%3A+release+0.2.1+available"></a>
+<a name="N10152"></a><a name="12+May%2C+2006%3A+release+0.2.1+available"></a>
 <h3 class="h4">12 May, 2006: release 0.2.1 available</h3>
 <p>This fixes a few bugs in release 0.2.0, listed in the <a href="http://tinyurl.com/rnnvz">change log</a>. The
       release can be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10152"></a><a name="5+May%2C+2006%3A+release+0.2.0+available"></a>
+<a name="N10164"></a><a name="5+May%2C+2006%3A+release+0.2.0+available"></a>
 <h3 class="h4">5 May, 2006: release 0.2.0 available</h3>
 <p>We are now aiming for monthly releases.  There have been many
       bug fixes and improvements in the past month.  MapReduce and DFS
@@ -322,24 +331,24 @@ document.write("<text>Last Published:</text> " + document.lastModified);
       details. The release can be obtained from <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
       nearby mirror</a>.
       </p>
-<a name="N10164"></a><a name="2+April%2C+2006%3A+release+0.1.0+available"></a>
+<a name="N10176"></a><a name="2+April%2C+2006%3A+release+0.1.0+available"></a>
 <h3 class="h4">2 April, 2006: release 0.1.0 available</h3>
 <p>This is the first Hadoop release.  The release is available
       <a href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/">
       here</a>.</p>
-<a name="N10172"></a><a name="6+February%2C+2006%3A+nightly+builds"></a>
+<a name="N10184"></a><a name="6+February%2C+2006%3A+nightly+builds"></a>
 <h3 class="h4">6 February, 2006: nightly builds</h3>
 <p>Hadoop now has nightly builds.  This automatically creates a
       <a href="http://cvs.apache.org/dist/lucene/hadoop/nightly/">downloadable version of Hadoop every
       night</a>.  All unit tests must pass, or a message is sent to
       the developers mailing list and no new version is created.  This
       also updates the <a href="api/index.html">javadoc</a>.</p>
-<a name="N10184"></a><a name="3+February%2C+2006%3A+Hadoop+code+moved+out+of+Nutch"></a>
+<a name="N10196"></a><a name="3+February%2C+2006%3A+Hadoop+code+moved+out+of+Nutch"></a>
 <h3 class="h4">3 February, 2006: Hadoop code moved out of Nutch</h3>
 <p>The Hadoop code has now been moved into its own Subversion
       tree, renamed into packages under <span class="codefrag">org.apache.hadoop</span>.
       All unit tests pass, but little else has yet been tested.</p>
-<a name="N10191"></a><a name="30+March%2C+2006%3A+Hadoop+project+approved"></a>
+<a name="N101A3"></a><a name="30+March%2C+2006%3A+Hadoop+project+approved"></a>
 <h3 class="h4">30 March, 2006: Hadoop project approved</h3>
 <p>The Lucene PMC has elected to split the Nutch MapReduce and
       distributed filesytem code into a new project named Hadoop.</p>

Різницю між файлами не показано, бо вона завелика
+ 22 - 22
docs/index.pdf


+ 9 - 0
src/docs/src/documentation/content/xdocs/index.xml

@@ -14,6 +14,15 @@
     <section>
       <title>News</title>
 
+      <section>
+      <title>10 January, 2007: release 0.10.1 available</title>
+      <p>This fixes critical bugs in 0.10.0.  For details see the <a
+      href="http://tinyurl.com/ycpsms">release notes</a>. The release can
+      be obtained from <a
+      href="http://www.apache.org/dyn/closer.cgi/lucene/hadoop/"> a
+      nearby mirror</a>.
+      </p> </section>
+
       <section>
       <title>5 January, 2007: release 0.10.0 available</title>
       <p>For details see the <a

+ 8 - 1
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -98,7 +98,14 @@ public class FileUtil {
         copyContent(in, out, conf);
       } finally {
         in.close();
-      } 
+      }
+      // if crc copying is disabled, remove the existing crc file if any
+      if (!copyCrc) {
+        Path crcFile = dstFS.getChecksumFile(dst);
+        if (dstFS.exists(crcFile)) {
+          dstFS.deleteRaw(crcFile);
+        }
+      }
     } else {
       throw new IOException(src.toString() + ": No such file or directory");
     }

+ 1 - 1
src/java/org/apache/hadoop/fs/s3/Jets3tFileSystemStore.java

@@ -216,7 +216,7 @@ class Jets3tFileSystemStore implements FileSystemStore {
     if (!path.isAbsolute()) {
       throw new IllegalArgumentException("Path must be absolute: " + path);
     }
-    return urlEncode(path.toString());
+    return urlEncode(path.toUri().getPath());
   }
 
   private Path keyToPath(String key) {

+ 27 - 13
src/java/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -83,12 +83,12 @@ public class S3FileSystem extends FileSystem {
     Path absolutePath = makeAbsolute(path);
     INode inode = store.getINode(absolutePath);
     if (inode == null) {
-      store.storeINode(path, INode.DIRECTORY_INODE);
+      store.storeINode(absolutePath, INode.DIRECTORY_INODE);
     } else if (inode.isFile()) {
       throw new IOException(String.format(
-          "Can't make directory for path %s since it is a file.", path));
+          "Can't make directory for path %s since it is a file.", absolutePath));
     }
-    Path parent = path.getParent();
+    Path parent = absolutePath.getParent();
     return (parent == null || mkdirs(parent));
   }
 
@@ -123,13 +123,14 @@ public class S3FileSystem extends FileSystem {
 
   @Override
   public Path[] listPathsRaw(Path path) throws IOException {
-    INode inode = store.getINode(makeAbsolute(path));
+    Path absolutePath = makeAbsolute(path);
+    INode inode = store.getINode(absolutePath);
     if (inode == null) {
       return null;
     } else if (inode.isFile()) {
-      return new Path[] { path };
+      return new Path[] { absolutePath };
     } else { // directory
-      Set<Path> paths = store.listSubPaths(path);
+      Set<Path> paths = store.listSubPaths(absolutePath);
       return paths.toArray(new Path[0]);
     }
   }
@@ -146,10 +147,6 @@ public class S3FileSystem extends FileSystem {
       short replication, long blockSize, Progressable progress)
       throws IOException {
 
-    if (!isDirectory(file.getParent())) {
-      throw new IOException("Cannot create file " + file
-          + " since parent directory does not exist.");
-    }
     INode inode = store.getINode(makeAbsolute(file));
     if (inode != null) {
       if (overwrite) {
@@ -157,6 +154,13 @@ public class S3FileSystem extends FileSystem {
       } else {
         throw new IOException("File already exists: " + file);
       }
+    } else {
+      Path parent = file.getParent();
+      if (parent != null) {
+        if (!mkdirs(parent)) {
+          throw new IOException("Mkdirs failed to create " + parent.toString());
+        }
+      }      
     }
     return new S3OutputStream(getConf(), store, makeAbsolute(file),
         blockSize, progress);
@@ -184,17 +188,27 @@ public class S3FileSystem extends FileSystem {
 
   @Override
   public boolean deleteRaw(Path path) throws IOException {
-    // TODO: Check if path is directory with children
     Path absolutePath = makeAbsolute(path);
     INode inode = store.getINode(absolutePath);
     if (inode == null) {
-      throw new IOException("No such file or directory.");
+      return false;
     }
-    store.deleteINode(absolutePath);
     if (inode.isFile()) {
+      store.deleteINode(absolutePath);
       for (Block block : inode.getBlocks()) {
         store.deleteBlock(block);
       }
+    } else {
+      Path[] contents = listPathsRaw(absolutePath);
+      if (contents == null) {
+        return false;
+      }
+      for (Path p : contents) {
+        if (! deleteRaw(p)) {
+          return false;
+        }
+      }
+      store.deleteINode(absolutePath);
     }
     return true;
   }

+ 114 - 0
src/java/org/apache/hadoop/io/compress/BlockCompressorStream.java

@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.compress;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * A {@link org.apache.hadoop.io.compress.CompressorStream} which works
+ * with 'block-based' based compression algorithms, as opposed to 
+ * 'stream-based' compression algorithms.
+ *  
+ * @author Arun C Murthy
+ */
+class BlockCompressorStream extends CompressorStream {
+
+  // The 'maximum' size of input data to be compressed, to account
+  // for the overhead of the compression algorithm.
+  private final int MAX_INPUT_SIZE;
+
+  /**
+   * Create a {@link BlockCompressorStream}.
+   * 
+   * @param out stream
+   * @param compressor compressor to be used
+   * @param bufferSize size of buffer
+   * @param compressionOverhead maximum 'overhead' of the compression 
+   *                            algorithm with given bufferSize
+   */
+  public BlockCompressorStream(OutputStream out, Compressor compressor, 
+      int bufferSize, int compressionOverhead) {
+    super(out, compressor, bufferSize);
+    MAX_INPUT_SIZE = bufferSize - compressionOverhead;
+  }
+
+  /**
+   * Create a {@link BlockCompressorStream} with given output-stream and 
+   * compressor.
+   * Use default of 512 as bufferSize and compressionOverhead of 
+   * (1% of bufferSize + 12 bytes) =  18 bytes (zlib algorithm).
+   * 
+   * @param out stream
+   * @param compressor compressor to be used
+   */
+  public BlockCompressorStream(OutputStream out, Compressor compressor) {
+    this(out, compressor, 512, 18);
+  }
+
+  public void write(byte[] b, int off, int len) throws IOException {
+    // Sanity checks
+    if (compressor.finished()) {
+      throw new IOException("write beyond end of stream");
+    }
+    if (b == null) {
+      throw new NullPointerException();
+    } else if ((off < 0) || (off > b.length) || (len < 0) ||
+            ((off + len) > b.length)) {
+      throw new IndexOutOfBoundsException();
+    } else if (len == 0) {
+      return;
+    }
+
+    // Write out the length of the original data
+    rawWriteInt(len);
+    
+    // Compress data
+    if (!compressor.finished()) {
+      do {
+        // Compress atmost 'maxInputSize' chunks at a time
+        int bufLen = Math.min(len, MAX_INPUT_SIZE);
+        
+        compressor.setInput(b, off, bufLen);
+        while (!compressor.needsInput()) {
+          compress();
+        }
+        off += bufLen;
+        len -= bufLen;
+      } while (len > 0);
+    }
+  }
+
+  void compress() throws IOException {
+    int len = compressor.compress(buffer, 0, buffer.length);
+    if (len > 0) {
+      // Write out the compressed chunk
+      rawWriteInt(len);
+      out.write(buffer, 0, len);
+    }
+  }
+  
+  private void rawWriteInt(int v) throws IOException {
+    out.write((v >>> 24) & 0xFF);
+    out.write((v >>> 16) & 0xFF);
+    out.write((v >>>  8) & 0xFF);
+    out.write((v >>>  0) & 0xFF);
+  }
+
+}

+ 129 - 0
src/java/org/apache/hadoop/io/compress/BlockDecompressorStream.java

@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.compress;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A {@link org.apache.hadoop.io.compress.DecompressorStream} which works
+ * with 'block-based' based compression algorithms, as opposed to 
+ * 'stream-based' compression algorithms.
+ *  
+ * @author Arun C Murthy
+ */
+class BlockDecompressorStream extends DecompressorStream {
+  private int originalBlockSize = 0;
+  private int noUncompressedBytes = 0;
+
+  /**
+   * Create a {@link BlockDecompressorStream}.
+   * 
+   * @param in input stream
+   * @param decompressor decompressor to use
+   * @param bufferSize size of buffer
+   */
+  public BlockDecompressorStream(InputStream in, Decompressor decompressor, 
+      int bufferSize) {
+    super(in, decompressor, bufferSize);
+  }
+  
+  /**
+   * Create a {@link BlockDecompressorStream}.
+   * 
+   * @param in input stream
+   * @param decompressor decompressor to use
+   */
+  public BlockDecompressorStream(InputStream in, Decompressor decompressor) {
+    super(in, decompressor);
+  }
+
+  protected BlockDecompressorStream(InputStream in) {
+    super(in);
+  }
+
+  int decompress(byte[] b, int off, int len) throws IOException {
+    // Check if we are the beginning of a block
+    if (noUncompressedBytes == originalBlockSize) {
+      // Get original data size
+      try {
+        originalBlockSize =  rawReadInt();
+      } catch (IOException ioe) {
+        return -1;
+      }
+      noUncompressedBytes = 0;
+    }
+    
+    int n = 0;
+    while ((n = decompressor.decompress(b, off, len)) == 0) {
+      if (decompressor.finished() || decompressor.needsDictionary()) {
+        if (noUncompressedBytes >= originalBlockSize) {
+          eof = true;
+          return -1;
+        }
+      }
+      if (decompressor.needsInput()) {
+        getCompressedData();
+      }
+    }
+    
+    // Note the no. of decompressed bytes read from 'current' block
+    noUncompressedBytes += n;
+
+    return n;
+  }
+
+  void getCompressedData() throws IOException {
+    checkStream();
+
+    // Get the size of the compressed chunk
+    int len = rawReadInt();
+
+    // Read len bytes from underlying stream 
+    if (len > buffer.length) {
+      buffer = new byte[len];
+    }
+    int n = 0, off = 0;
+    while (n < len) {
+      int count = in.read(buffer, off + n, len - n);
+      if (count < 0) {
+        throw new EOFException();
+      }
+      n += count;
+    }
+    
+    // Send the read data to the decompressor
+    decompressor.setInput(buffer, 0, len);
+  }
+
+  public void resetState() throws IOException {
+    super.resetState();
+  }
+
+  private int rawReadInt() throws IOException {
+    int b1 = in.read();
+    int b2 = in.read();
+    int b3 = in.read();
+    int b4 = in.read();
+    if ((b1 | b2 | b3 | b4) < 0)
+        throw new EOFException();
+    return ((b1 << 24) + (b2 << 16) + (b3 << 8) + (b4 << 0));
+  }
+}

+ 157 - 0
src/java/org/apache/hadoop/io/compress/LzoCodec.java

@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.compress;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.InputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.compress.lzo.*;
+import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A {@link org.apache.hadoop.io.compress.CompressionCodec} for a streaming
+ * <b>lzo</b> compression/decompression pair.
+ * http://www.oberhumer.com/opensource/lzo/
+ * 
+ * @author Arun C Murthy
+ */
+public class LzoCodec implements Configurable, CompressionCodec {
+  
+  private static final Log LOG = LogFactory.getLog(LzoCodec.class.getName());
+
+  private Configuration conf;
+  
+  public void setConf(Configuration conf) {
+	  this.conf = conf;
+  }
+  
+  public Configuration getConf() {
+	  return conf;
+  }
+
+  private static boolean nativeLzoLoaded = false;
+  
+  static {
+    if (NativeCodeLoader.isNativeCodeLoaded()) {
+      nativeLzoLoaded = LzoCompressor.isNativeLzoLoaded() &&
+                          LzoDecompressor.isNativeLzoLoaded();
+      
+      if (nativeLzoLoaded) {
+        LOG.info("Successfully loaded & initialized native-lzo library");
+      } else {
+        LOG.error("Failed to load/initialize native-lzo library");
+      }
+    } else {
+      LOG.error("Cannot load native-lzo without native-hadoop");
+    }
+  }
+
+  /**
+   * Check if native-lzo library is loaded & initialized.
+   * 
+   * @return <code>true</code> if native-lzo library is loaded & initialized;
+   *         else <code>false</code>
+   */
+  public static boolean isNativeLzoLoaded() {
+    return nativeLzoLoaded;
+  }
+  
+  public CompressionOutputStream createOutputStream(OutputStream out) 
+  throws IOException {
+    // Ensure native-lzo library is loaded & initialized
+    if (!isNativeLzoLoaded()) {
+      throw new IOException("native-lzo library not available");
+    }
+    
+    /**
+     * <b>http://www.oberhumer.com/opensource/lzo/lzofaq.php</b>
+     *
+     * How much can my data expand during compression ?
+     * ================================================
+     * LZO will expand incompressible data by a little amount.
+     * I still haven't computed the exact values, but I suggest using
+     * these formulas for a worst-case expansion calculation:
+     * 
+     * Algorithm LZO1, LZO1A, LZO1B, LZO1C, LZO1F, LZO1X, LZO1Y, LZO1Z:
+     * ----------------------------------------------------------------
+     * output_block_size = input_block_size + (input_block_size / 16) + 64 + 3
+     * 
+     * This is about 106% for a large block size.
+     * 
+     * Algorithm LZO2A:
+     * ----------------
+     * output_block_size = input_block_size + (input_block_size / 8) + 128 + 3
+     */
+
+    // Create the lzo output-stream
+    LzoCompressor.CompressionStrategy strategy = 
+      LzoCompressor.CompressionStrategy.valueOf(
+              conf.get("io.compression.codec.lzo.compressor",
+                        LzoCompressor.CompressionStrategy.LZO1X_1.name()
+                      )
+                    ); 
+    int bufferSize = conf.getInt("io.compression.codec.lzo.buffersize", 
+                                  64*1024);
+    int compressionOverhead = 0;
+    if (strategy.name().contains("LZO1")) {
+      compressionOverhead = (int)(((bufferSize - (64 + 3)) * 16.0) / 17.0);  
+    } else {
+      compressionOverhead = (int)(((bufferSize - (128 + 3)) * 8.0) / 9.0);
+    }
+     
+    return new BlockCompressorStream(out, 
+            new LzoCompressor(strategy, bufferSize), 
+            bufferSize, compressionOverhead);
+  }
+  
+  public CompressionInputStream createInputStream(InputStream in) 
+  throws IOException {
+    // Ensure native-lzo library is loaded & initialized
+    if (!isNativeLzoLoaded()) {
+      throw new IOException("native-lzo library not available");
+    }
+    
+    // Create the lzo input-stream
+    LzoDecompressor.CompressionStrategy strategy = 
+      LzoDecompressor.CompressionStrategy.valueOf(
+              conf.get("io.compression.codec.lzo.decompressor",
+                        LzoDecompressor.CompressionStrategy.LZO1X.name()
+                      )
+                    ); 
+    int bufferSize = conf.getInt("io.compression.codec.lzo.buffersize", 
+                                  64*1024);
+
+    return new BlockDecompressorStream(in, 
+            new LzoDecompressor(strategy, bufferSize), 
+            bufferSize);
+  }
+  
+  /**
+   * Get the default filename extension for this kind of compression.
+   * @return the extension including the '.'
+   */
+  public String getDefaultExtension() {
+    return ".lzo";
+  }
+}

+ 319 - 0
src/java/org/apache/hadoop/io/compress/lzo/LzoCompressor.java

@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.compress.lzo;
+
+import java.io.IOException;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+/**
+ * A {@link Compressor} based on the lzo algorithm.
+ * http://www.oberhumer.com/opensource/lzo/
+ * 
+ * @author Arun C Murthy
+ */
+public class LzoCompressor implements Compressor {
+  private static final Log LOG = 
+    LogFactory.getLog(LzoCompressor.class.getName());
+
+  private int directBufferSize;
+  private byte[] userBuf = null;
+  private int userBufOff = 0, userBufLen = 0;
+  private Buffer uncompressedDirectBuf = null;
+  private int uncompressedDirectBufLen = 0;
+  private Buffer compressedDirectBuf = null;
+  private boolean finish, finished;
+  
+  private CompressionStrategy strategy; // The lzo compression algorithm.
+  private long lzoCompressor = 0;       // The actual lzo compression function.
+  private int workingMemoryBufLen = 0;  // The length of 'working memory' buf.
+  private Buffer workingMemoryBuf;      // The 'working memory' for lzo.
+  
+  /**
+   * The compression algorithm for lzo library.
+   */
+  public static enum CompressionStrategy {
+    /**
+     * lzo1 algorithms.
+     */
+    LZO1 (0),
+    LZO1_99 (1),
+    
+    /**
+     * lzo1a algorithms.
+     */
+    LZO1A (2),
+    LZO1A_99 (3),
+    
+    /**
+     * lzo1b algorithms.
+     */
+    LZO1B (4),
+    LZO1B_BEST_COMPRESSION(5),
+    LZO1B_BEST_SPEED(6),
+    LZO1B_1 (7),
+    LZO1B_2 (8),
+    LZO1B_3 (9),
+    LZO1B_4 (10),
+    LZO1B_5 (11),
+    LZO1B_6 (12),
+    LZO1B_7 (13),
+    LZO1B_8 (14),
+    LZO1B_9 (15),
+    LZO1B_99 (16),
+    LZO1B_999 (17),
+
+    /**
+     * lzo1c algorithms.
+     */
+    LZO1C (18),
+    LZO1C_BEST_COMPRESSION(19),
+    LZO1C_BEST_SPEED(20),
+    LZO1C_1 (21),
+    LZO1C_2 (22),
+    LZO1C_3 (23),
+    LZO1C_4 (24),
+    LZO1C_5 (25),
+    LZO1C_6 (26),
+    LZO1C_7 (27),
+    LZO1C_8 (28),
+    LZO1C_9 (29),
+    LZO1C_99 (30),
+    LZO1C_999 (31),
+    
+    /**
+     * lzo1f algorithms.
+     */
+    LZO1F_1 (32),
+    LZO1F_999 (33),
+    
+    /**
+     * lzo1x algorithms.
+     */
+    LZO1X_1 (34),
+    LZO1X_11 (35),
+    LZO1X_12 (36),
+    LZO1X_15 (37),
+    LZO1X_999 (38),
+    
+    /**
+     * lzo1y algorithms.
+     */
+    LZO1Y_1 (39),
+    LZO1Y_999 (40),
+    
+    /**
+     * lzo1z algorithms.
+     */
+    LZO1Z_999 (41),
+    
+    /**
+     * lzo2a algorithms.
+     */
+    LZO2A_999 (42);
+    
+    private final int compressor;
+
+    private CompressionStrategy(int compressor) {
+      this.compressor = compressor;
+    }
+    
+    int getCompressor() {
+      return compressor;
+    }
+  }; // CompressionStrategy
+
+  private static boolean nativeLzoLoaded = false;
+  
+  static {
+    if (NativeCodeLoader.isNativeCodeLoaded()) {
+      // Initialize the native library
+      initIDs();
+      nativeLzoLoaded = true;
+    } else {
+      LOG.error("Cannot load " + LzoCompressor.class.getName() + 
+              " without native-hadoop library!");
+    }
+  }
+  
+  /**
+   * Check if lzo compressors are loaded and initialized.
+   * 
+   * @return <code>true</code> if lzo compressors are loaded & initialized,
+   *         else <code>false</code> 
+   */
+  public static boolean isNativeLzoLoaded() {
+    return nativeLzoLoaded;
+  }
+
+  /** 
+   * Creates a new compressor using the specified {@link CompressionStrategy}.
+   * 
+   * @param strategy lzo compression algorithm to use
+   * @param directBufferSize size of the direct buffer to be used.
+   */
+  public LzoCompressor(CompressionStrategy strategy, int directBufferSize) {
+    this.strategy = strategy;
+    this.directBufferSize = directBufferSize;
+    uncompressedDirectBuf = ByteBuffer.allocateDirect(directBufferSize);
+    compressedDirectBuf = ByteBuffer.allocateDirect(directBufferSize);
+    compressedDirectBuf.position(directBufferSize);
+    
+    /**
+     * Initialize {@link #lzoCompress} and {@link #workingMemoryBufLen}
+     */
+    init(this.strategy.getCompressor());
+    workingMemoryBuf = ByteBuffer.allocateDirect(workingMemoryBufLen);
+  }
+  
+  /**
+   * Creates a new compressor with the default lzo1x_1 compression.
+   */
+  public LzoCompressor() {
+    this(CompressionStrategy.LZO1X_1, 64*1024);
+  }
+  
+  public synchronized void setInput(byte[] b, int off, int len) {
+    if (b== null) {
+      throw new NullPointerException();
+    }
+    if (off < 0 || len < 0 || off > b.length - len) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+
+    this.userBuf = b;
+    this.userBufOff = off;
+    this.userBufLen = len;
+
+    // Reinitialize lzo's output direct-buffer 
+    compressedDirectBuf.limit(directBufferSize);
+    compressedDirectBuf.position(directBufferSize);
+  }
+
+  synchronized void setInputFromSavedData() {
+    uncompressedDirectBufLen = userBufLen;
+    if (uncompressedDirectBufLen > directBufferSize) {
+      uncompressedDirectBufLen = directBufferSize;
+    }
+
+    // Reinitialize lzo's input direct buffer
+    uncompressedDirectBuf.rewind();
+    ((ByteBuffer)uncompressedDirectBuf).put(userBuf, userBufOff,  
+                                          uncompressedDirectBufLen);
+
+    // Note how much data is being fed to lzo
+    userBufOff += uncompressedDirectBufLen;
+    userBufLen -= uncompressedDirectBufLen;
+  }
+
+  public synchronized void setDictionary(byte[] b, int off, int len) {
+    // nop
+  }
+
+  public boolean needsInput() {
+    // Consume remaining compressed data?
+    if (compressedDirectBuf.remaining() > 0) {
+      return false;
+    }
+
+    // Check if lzo has consumed all input
+    if (uncompressedDirectBufLen <= 0) {
+      // Check if we have consumed all user-input
+      if (userBufLen <= 0) {
+        return true;
+      } else {
+        setInputFromSavedData();
+      }
+    }
+    
+    return false;
+  }
+  
+  public synchronized void finish() {
+    finish = true;
+  }
+  
+  public synchronized boolean finished() {
+    // Check if 'lzo' says its 'finished' and
+    // all compressed data has been consumed
+    return (finished && compressedDirectBuf.remaining() == 0); 
+  }
+
+  public synchronized int compress(byte[] b, int off, int len) 
+  throws IOException {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    if (off < 0 || len < 0 || off > b.length - len) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+    
+    int n = 0;
+    
+    // Check if there is compressed data
+    n = compressedDirectBuf.remaining();
+    if (n > 0) {
+      n = Math.min(n, len);
+      ((ByteBuffer)compressedDirectBuf).get(b, off, n);
+      return n;
+    }
+
+    // Re-initialize the lzo's output direct-buffer
+    compressedDirectBuf.rewind();
+    compressedDirectBuf.limit(directBufferSize);
+
+    // Compress data
+    n = compressBytesDirect(strategy.getCompressor());
+    compressedDirectBuf.limit(n);
+    
+    // Set 'finished' if lzo has consumed all user-data
+    if (userBufLen <= 0) {
+      finished = true;
+    }
+    
+    // Get atmost 'len' bytes
+    n = Math.min(n, len);
+    ((ByteBuffer)compressedDirectBuf).get(b, off, n);
+
+    return n;
+  }
+
+  public synchronized void reset() {
+    finish = false;
+    finished = false;
+    uncompressedDirectBuf.rewind();
+    uncompressedDirectBufLen = 0;
+    compressedDirectBuf.limit(directBufferSize);
+    compressedDirectBuf.position(directBufferSize);
+    userBufOff = userBufLen = 0;
+  }
+  
+  public synchronized void end() {
+    // nop
+  }
+  
+  private native static void initIDs();
+  private native void init(int compressor);
+  private native int compressBytesDirect(int compressor);
+}

+ 308 - 0
src/java/org/apache/hadoop/io/compress/lzo/LzoDecompressor.java

@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.compress.lzo;
+
+import java.io.IOException;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+/**
+ * A {@link Decompressor} based on the lzo algorithm.
+ * http://www.oberhumer.com/opensource/lzo/
+ * 
+ * @author Arun C Murthy
+ */
+public class LzoDecompressor implements Decompressor {
+  private static final Log LOG = 
+    LogFactory.getLog(LzoDecompressor.class.getName());
+  
+  private int directBufferSize;
+  private Buffer compressedDirectBuf = null;
+  private int compressedDirectBufLen;
+  private Buffer uncompressedDirectBuf = null;
+  private byte[] userBuf = null;
+  private int userBufOff = 0, userBufLen = 0;
+  private boolean finished;
+  
+  private CompressionStrategy strategy;
+  private long lzoDecompressor = 0;   // The actual lzo decompression function.
+  
+  public static enum CompressionStrategy {
+    /**
+     * lzo1 algorithms.
+     */
+    LZO1 (0),
+
+    /**
+     * lzo1a algorithms.
+     */
+    LZO1A (1),
+
+    /**
+     * lzo1b algorithms.
+     */
+    LZO1B (2),
+    LZO1B_SAFE(3),
+
+    /**
+     * lzo1c algorithms.
+     */
+    LZO1C (4),
+    LZO1C_SAFE(5),
+    LZO1C_ASM (6),
+    LZO1C_ASM_SAFE (7),
+
+    /**
+     * lzo1f algorithms.
+     */
+    LZO1F (8),
+    LZO1F_SAFE (9),
+    LZO1F_ASM_FAST (10),
+    LZO1F_ASM_FAST_SAFE (11),
+    
+    /**
+     * lzo1x algorithms.
+     */
+    LZO1X (12),
+    LZO1X_SAFE (13),
+    LZO1X_ASM (14),
+    LZO1X_ASM_SAFE (15),
+    LZO1X_ASM_FAST (16),
+    LZO1X_ASM_FAST_SAFE (17),
+    
+    /**
+     * lzo1y algorithms.
+     */
+    LZO1Y (18),
+    LZO1Y_SAFE (19),
+    LZO1Y_ASM (20),
+    LZO1Y_ASM_SAFE (21),
+    LZO1Y_ASM_FAST (22),
+    LZO1Y_ASM_FAST_SAFE (23),
+    
+    /**
+     * lzo1z algorithms.
+     */
+    LZO1Z (24),
+    LZO1Z_SAFE (25),
+    
+    /**
+     * lzo2a algorithms.
+     */
+    LZO2A (26),
+    LZO2A_SAFE (27);
+    
+    private final int decompressor;
+
+    private CompressionStrategy(int decompressor) {
+      this.decompressor = decompressor;
+    }
+    
+    int getDecompressor() {
+      return decompressor;
+    }
+  }; // CompressionStrategy
+  
+  private static boolean nativeLzoLoaded = false;
+  
+  static {
+    if (NativeCodeLoader.isNativeCodeLoaded()) {
+      // Initialize the native library
+      initIDs();
+      nativeLzoLoaded = true;
+    } else {
+      LOG.error("Cannot load " + LzoDecompressor.class.getName() + 
+              " without native-hadoop library!");
+    }
+  }
+  
+  /**
+   * Check if lzo decompressors are loaded and initialized.
+   * 
+   * @return <code>true</code> if lzo decompressors are loaded & initialized,
+   *         else <code>false</code> 
+   */
+  public static boolean isNativeLzoLoaded() {
+    return nativeLzoLoaded;
+  }
+
+  /**
+   * Creates a new lzo decompressor.
+   * 
+   * @param strategy lzo decompression algorithm
+   * @param directBufferSize size of the direct-buffer
+   */
+  public LzoDecompressor(CompressionStrategy strategy, int directBufferSize) {
+    this.directBufferSize = directBufferSize;
+    this.strategy = strategy;
+    
+    compressedDirectBuf = ByteBuffer.allocateDirect(directBufferSize);
+    uncompressedDirectBuf = ByteBuffer.allocateDirect(directBufferSize);
+    uncompressedDirectBuf.position(directBufferSize);
+    
+    /**
+     * Initialize {@link #lzoDecompress}
+     */
+    init(this.strategy.getDecompressor());
+  }
+  
+  /**
+   * Creates a new lzo decompressor.
+   */
+  public LzoDecompressor() {
+    this(CompressionStrategy.LZO1X, 64*1024);
+  }
+
+  public synchronized void setInput(byte[] b, int off, int len) {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    if (off < 0 || len < 0 || off > b.length - len) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+  
+    this.userBuf = b;
+    this.userBufOff = off;
+    this.userBufLen = len;
+    
+    setInputFromSavedData();
+    
+    // Reinitialize lzo's output direct-buffer 
+    uncompressedDirectBuf.limit(directBufferSize);
+    uncompressedDirectBuf.position(directBufferSize);
+  }
+  
+  synchronized void setInputFromSavedData() {
+    compressedDirectBufLen = userBufLen;
+    if (compressedDirectBufLen > directBufferSize) {
+      compressedDirectBufLen = directBufferSize;
+    }
+
+    // Reinitialize lzo's input direct-buffer
+    compressedDirectBuf.rewind();
+    ((ByteBuffer)compressedDirectBuf).put(userBuf, userBufOff, 
+                                        compressedDirectBufLen);
+    
+    // Note how much data is being fed to lzo
+    userBufOff += compressedDirectBufLen;
+    userBufLen -= compressedDirectBufLen;
+  }
+
+  public synchronized void setDictionary(byte[] b, int off, int len) {
+    // nop
+  }
+
+  public synchronized boolean needsInput() {
+    // Consume remanining compressed data?
+    if (uncompressedDirectBuf.remaining() > 0) {
+      return false;
+    }
+    
+    // Check if lzo has consumed all input
+    if (compressedDirectBufLen <= 0) {
+      // Check if we have consumed all user-input
+      if (userBufLen <= 0) {
+        return true;
+      } else {
+        setInputFromSavedData();
+      }
+    }
+    
+    return false;
+  }
+
+  public synchronized boolean needsDictionary() {
+    return false;
+  }
+
+  public synchronized boolean finished() {
+    // Check if 'lzo' says its 'finished' and
+    // all uncompressed data has been consumed
+    return (finished && uncompressedDirectBuf.remaining() == 0);
+  }
+
+  public synchronized int decompress(byte[] b, int off, int len) 
+  throws IOException {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    if (off < 0 || len < 0 || off > b.length - len) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+    
+    int n = 0;
+    
+    // Check if there is uncompressed data
+    n = uncompressedDirectBuf.remaining();
+    if(n > 0) {
+      n = Math.min(n, len);
+      ((ByteBuffer)uncompressedDirectBuf).get(b, off, n);
+      return n;
+    }
+    
+    // Check if there is data to decompress
+    if (compressedDirectBufLen <= 0) {
+      return 0;
+    }
+    
+    // Re-initialize the lzo's output direct-buffer
+    uncompressedDirectBuf.rewind();
+    uncompressedDirectBuf.limit(directBufferSize);
+
+    // Decompress data
+    n = decompressBytesDirect(strategy.getDecompressor());
+    uncompressedDirectBuf.limit(n);
+
+    // Set 'finished' if lzo has consumed all user-data
+    if (userBufLen <= 0) {
+      finished = true;
+    }
+    
+    // Return atmost 'len' bytes
+    n = Math.min(n, len);
+    ((ByteBuffer)uncompressedDirectBuf).get(b, off, n);
+
+    return n;
+  }
+  
+  public synchronized void reset() {
+    finished = false;
+    compressedDirectBufLen = 0;
+    uncompressedDirectBuf.limit(directBufferSize);
+    uncompressedDirectBuf.position(directBufferSize);
+    userBufOff = userBufLen = 0;
+  }
+
+  public synchronized void end() {
+    // nop
+  }
+
+  protected void finalize() {
+    end();
+  }
+  
+  private native static void initIDs();
+  private native void init(int decompressor);
+  private native int decompressBytesDirect(int decompressor);
+}

+ 2 - 2
src/java/org/apache/hadoop/mapred/InputFormatBase.java

@@ -108,9 +108,9 @@ public abstract class InputFormatBase implements InputFormat {
         // while generating splits
         for (Path subPath : fs.listPaths(p, hiddenFileFilter)) {
           FileSystem subFS = subPath.getFileSystem(job); 
-          if (!subFS.isFile(subPath)) {
+          if (!subFS.exists(subPath)) {
             result.add(new IOException(
-                "Input path is not a file : " + subPath)); 
+                "Input path does not exist: " + subPath)); 
           } else {
             totalFiles++; 
           }

+ 55 - 30
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -454,40 +454,52 @@ class JobInProgress {
                                            TaskStatus status,
                                            JobTrackerMetrics metrics) {
         String taskid = status.getTaskId();
+        
+        // Sanity check: is the TIP already complete?
         if (tip.isComplete()) {
           LOG.info("Already complete TIP " + tip.getTIPId() + 
-                   " has completed task " + taskid);
-          return;
-        } else {
-          LOG.info("Task '" + taskid + "' has completed " + tip.getTIPId() + 
-                   " successfully.");          
-
-          String taskTrackerName = status.getTaskTracker();
+               " has completed task " + taskid);
+          
+          // Just mark this 'task' as complete
+          tip.completedTask(taskid);
           
-          if(status.getIsMap()){
-            JobHistory.MapAttempt.logStarted(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
-                taskTrackerName); 
-            JobHistory.MapAttempt.logFinished(profile.getJobId(), 
-                tip.getTIPId(), status.getTaskId(), status.getFinishTime(), 
-                taskTrackerName); 
-            JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(), 
-                Values.MAP.name(), status.getFinishTime()); 
-          }else{
-              JobHistory.ReduceAttempt.logStarted(profile.getJobId(), 
-                  tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
-                  taskTrackerName); 
-              JobHistory.ReduceAttempt.logFinished(profile.getJobId(), 
-                  tip.getTIPId(), status.getTaskId(), status.getShuffleFinishTime(),
-                  status.getSortFinishTime(), status.getFinishTime(), 
-                  taskTrackerName); 
-              JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(), 
-                  Values.REDUCE.name(), status.getFinishTime()); 
+          // Let the JobTracker cleanup this taskid if the job isn't running
+          if (this.status.getRunState() != JobStatus.RUNNING) {
+            jobtracker.markCompletedTaskAttempt(status.getTaskTracker(), taskid);
           }
+          return;
+        } 
+
+        LOG.info("Task '" + taskid + "' has completed " + tip.getTIPId() + 
+          " successfully.");          
+
+        // Update jobhistory 
+        String taskTrackerName = status.getTaskTracker();
+        if(status.getIsMap()){
+          JobHistory.MapAttempt.logStarted(profile.getJobId(), 
+               tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
+               taskTrackerName); 
+          JobHistory.MapAttempt.logFinished(profile.getJobId(), 
+               tip.getTIPId(), status.getTaskId(), status.getFinishTime(), 
+               taskTrackerName); 
+          JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(), 
+               Values.MAP.name(), status.getFinishTime()); 
+        }else{
+          JobHistory.ReduceAttempt.logStarted(profile.getJobId(), 
+               tip.getTIPId(), status.getTaskId(), status.getStartTime(), 
+               taskTrackerName); 
+          JobHistory.ReduceAttempt.logFinished(profile.getJobId(), 
+               tip.getTIPId(), status.getTaskId(), status.getShuffleFinishTime(),
+               status.getSortFinishTime(), status.getFinishTime(), 
+               taskTrackerName); 
+          JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(), 
+               Values.REDUCE.name(), status.getFinishTime()); 
         }
         
+        // Mark the TIP as complete
         tip.completed(taskid);
-        // updating the running/finished map/reduce counts
+        
+        // Update the running/finished map/reduce counts
         if (tip.isMapTask()){
           runningMapTasks -= 1;
           finishedMapTasks += 1;
@@ -533,6 +545,10 @@ class JobInProgress {
             JobHistory.JobInfo.logFinished(this.status.getJobId(), finishTime, 
                 this.finishedMapTasks, this.finishedReduceTasks, failedMapTasks, failedReduceTasks);
             metrics.completeJob();
+        } else if (this.status.getRunState() != JobStatus.RUNNING) {
+            // The job has been killed/failed, 
+            // JobTracker should cleanup this task
+            jobtracker.markCompletedTaskAttempt(status.getTaskTracker(), taskid);
         }
     }
 
@@ -541,6 +557,7 @@ class JobInProgress {
      */
     public synchronized void kill() {
         if (status.getRunState() != JobStatus.FAILED) {
+            LOG.info("Killing job '" + this.status.getJobId() + "'");
             this.status = new JobStatus(status.getJobId(), 1.0f, 1.0f, JobStatus.FAILED);
             this.finishTime = System.currentTimeMillis();
             this.runningMapTasks = 0;
@@ -575,7 +592,9 @@ class JobInProgress {
     private void failedTask(TaskInProgress tip, String taskid, 
                             TaskStatus status, String trackerName,
                             boolean wasRunning, boolean wasComplete) {
+        // Mark the taskid as a 'failure'
         tip.failedSubTask(taskid, trackerName);
+        
         boolean isRunning = tip.isRunning();
         boolean isComplete = tip.isComplete();
         
@@ -621,6 +640,11 @@ class JobInProgress {
           failedReduceTasks++; 
         }
             
+        //
+        // Let the JobTracker know that this task has failed
+        //
+        jobtracker.markCompletedTaskAttempt(status.getTaskTracker(), taskid);
+
         //
         // Check if we need to kill the job because of too many failures
         //
@@ -633,9 +657,7 @@ class JobInProgress {
                 System.currentTimeMillis(), this.finishedMapTasks, this.finishedReduceTasks);
             kill();
         }
-
-        jobtracker.removeTaskEntry(taskid);
- }
+    }
 
     /**
      * Fail a task with a given reason, but without a status object.
@@ -669,6 +691,9 @@ class JobInProgress {
      * from the various tables.
      */
     synchronized void garbageCollect() {
+      // Let the JobTracker know that a job is complete
+      jobtracker.finalizeJob(this);
+      
       try {
         // Definitely remove the local-disk copy of the job file
         if (localJobFile != null) {

+ 314 - 64
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -46,6 +46,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     static float PAD_FRACTION;
     static final int MIN_CLUSTER_SIZE_FOR_PADDING = 3;
 
+    /**
+     * The maximum no. of 'completed' (successful/failed/killed)
+     * jobs kept in memory per-user. 
+     */
+    static final int MAX_COMPLETE_USER_JOBS_IN_MEMORY = 100;
+    
     /**
      * Used for formatting the id numbers
      */
@@ -215,36 +221,45 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                 //
                 // Loop through all expired items in the queue
                 //
-                synchronized (taskTrackers) {
+                // Need to lock the JobTracker here since we are
+                // manipulating it's data-structures via
+                // ExpireTrackers.run -> JobTracker.lostTaskTracker ->
+                // JobInProgress.failedTask -> JobTracker.markCompleteTaskAttempt
+                // Also need to lock JobTracker before locking 'taskTracker' &
+                // 'trackerExpiryQueue' to prevent deadlock:
+                // @see {@link JobTracker.processHeartbeat(TaskTrackerStatus, boolean)} 
+                synchronized (JobTracker.this) {
+                  synchronized (taskTrackers) {
                     synchronized (trackerExpiryQueue) {
-                        long now = System.currentTimeMillis();
-                        TaskTrackerStatus leastRecent = null;
-                        while ((trackerExpiryQueue.size() > 0) &&
-                               ((leastRecent = (TaskTrackerStatus) trackerExpiryQueue.first()) != null) &&
-                               (now - leastRecent.getLastSeen() > TASKTRACKER_EXPIRY_INTERVAL)) {
-
-                            // Remove profile from head of queue
-                            trackerExpiryQueue.remove(leastRecent);
-                            String trackerName = leastRecent.getTrackerName();
-
-                            // Figure out if last-seen time should be updated, or if tracker is dead
-                            TaskTrackerStatus newProfile = (TaskTrackerStatus) taskTrackers.get(leastRecent.getTrackerName());
-                            // Items might leave the taskTracker set through other means; the
-                            // status stored in 'taskTrackers' might be null, which means the
-                            // tracker has already been destroyed.
-                            if (newProfile != null) {
-                                if (now - newProfile.getLastSeen() > TASKTRACKER_EXPIRY_INTERVAL) {
-                                    // Remove completely
-                                    updateTaskTrackerStatus(trackerName, null);
-                                    lostTaskTracker(leastRecent.getTrackerName(),
-                                                    leastRecent.getHost());
-                                } else {
-                                    // Update time by inserting latest profile
-                                    trackerExpiryQueue.add(newProfile);
-                                }
-                            }
+                      long now = System.currentTimeMillis();
+                      TaskTrackerStatus leastRecent = null;
+                      while ((trackerExpiryQueue.size() > 0) &&
+                              ((leastRecent = (TaskTrackerStatus) trackerExpiryQueue.first()) != null) &&
+                              (now - leastRecent.getLastSeen() > TASKTRACKER_EXPIRY_INTERVAL)) {
+                        
+                        // Remove profile from head of queue
+                        trackerExpiryQueue.remove(leastRecent);
+                        String trackerName = leastRecent.getTrackerName();
+                        
+                        // Figure out if last-seen time should be updated, or if tracker is dead
+                        TaskTrackerStatus newProfile = (TaskTrackerStatus) taskTrackers.get(leastRecent.getTrackerName());
+                        // Items might leave the taskTracker set through other means; the
+                        // status stored in 'taskTrackers' might be null, which means the
+                        // tracker has already been destroyed.
+                        if (newProfile != null) {
+                          if (now - newProfile.getLastSeen() > TASKTRACKER_EXPIRY_INTERVAL) {
+                            // Remove completely
+                            updateTaskTrackerStatus(trackerName, null);
+                            lostTaskTracker(leastRecent.getTrackerName(),
+                                    leastRecent.getHost());
+                          } else {
+                            // Update time by inserting latest profile
+                            trackerExpiryQueue.add(newProfile);
+                          }
                         }
+                      }
                     }
+                  }
                 }
               } catch (Exception t) {
                 LOG.error("Tracker Expiry Thread got exception: " +
@@ -289,10 +304,26 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                                 if (job.getStatus().getRunState() != JobStatus.RUNNING &&
                                     job.getStatus().getRunState() != JobStatus.PREP &&
                                     (job.getFinishTime() + RETIRE_JOB_INTERVAL < System.currentTimeMillis())) {
+                                    // Ok, this call to removeTaskEntries
+                                    // is dangerous in some very very obscure
+                                    // cases; e.g. when job completed, exceeded
+                                    // RETIRE_JOB_INTERVAL time-limit and yet
+                                    // some task (taskid) wasn't complete!
+                                    removeJobTasks(job);
+                                    
                                     it.remove();
-                            
+                                    synchronized (userToJobsMap) {
+                                        ArrayList<JobInProgress> userJobs =
+                                            userToJobsMap.get(job.getProfile().getUser());
+                                        synchronized (userJobs) {
+                                            userJobs.remove(job);
+                                        }
+                                    }
                                     jobInitQueue.remove(job);
                                     jobsByArrival.remove(job);
+                                    
+                                    LOG.info("Retired job with id: '" + 
+                                            job.getProfile().getJobId() + "'");
                                 }
                             }
                         }
@@ -418,6 +449,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     TreeMap jobs = new TreeMap();
     Vector jobsByArrival = new Vector();
 
+    // (user -> list of JobInProgress)
+    TreeMap<String, ArrayList<JobInProgress>> userToJobsMap = new TreeMap();
+    
     // All the known TaskInProgress items, mapped to by taskids (taskid->TIP)
     Map<String, TaskInProgress> taskidToTIPMap = new TreeMap();
 
@@ -427,8 +461,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     // (trackerID->TreeSet of taskids running at that tracker)
     TreeMap trackerToTaskMap = new TreeMap();
 
-    // (trackerID --> last sent HeartBeatResponseID)
-    Map<String, Short> trackerToHeartbeatResponseIDMap = new TreeMap();
+    // (trackerID -> TreeSet of completed taskids running at that tracker)
+    TreeMap<String, Set<String>> trackerToMarkedTasksMap = new TreeMap();
+
+    // (trackerID --> last sent HeartBeatResponse)
+    Map<String, HeartbeatResponse> trackerToHeartbeatResponseMap = 
+      new TreeMap();
     
     //
     // Watch and expire TaskTracker objects using these structures.
@@ -644,18 +682,181 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         // taskid --> TIP
         taskidToTIPMap.put(taskid, tip);
     }
+    
     void removeTaskEntry(String taskid) {
         // taskid --> tracker
         String tracker = (String) taskidToTrackerMap.remove(taskid);
 
         // tracker --> taskid
-        TreeSet trackerSet = (TreeSet) trackerToTaskMap.get(tracker);
-        if (trackerSet != null) {
-            trackerSet.remove(taskid);
+        if (tracker != null) {
+            TreeSet trackerSet = (TreeSet) trackerToTaskMap.get(tracker);
+            if (trackerSet != null) {
+                trackerSet.remove(taskid);
+            }
         }
 
         // taskid --> TIP
         taskidToTIPMap.remove(taskid);
+        
+        LOG.debug("Removing task '" + taskid + "'");
+    }
+    
+    /**
+     * Mark a 'task' for removal later.
+     * This function assumes that the JobTracker is locked on entry.
+     * 
+     * @param taskTracker the tasktracker at which the 'task' was running
+     * @param taskid completed (success/failure/killed) task
+     */
+    void markCompletedTaskAttempt(String taskTracker, String taskid) {
+      // tracker --> taskid
+      TreeSet taskset = (TreeSet) trackerToMarkedTasksMap.get(taskTracker);
+      if (taskset == null) {
+        taskset = new TreeSet();
+        trackerToMarkedTasksMap.put(taskTracker, taskset);
+      }
+      taskset.add(taskid);
+      
+      LOG.debug("Marked '" + taskid + "' from '" + taskTracker + "'");
+    }
+
+    /**
+     * Mark all 'non-running' jobs of the job for pruning.
+     * This function assumes that the JobTracker is locked on entry.
+     * 
+     * @param job the completed job
+     */
+    void markCompletedJob(JobInProgress job) {
+      for (TaskInProgress tip : job.getMapTasks()) {
+        for (TaskStatus taskStatus : tip.getTaskStatuses()) {
+          if (taskStatus.getRunState() != TaskStatus.State.RUNNING) {
+            markCompletedTaskAttempt(taskStatus.getTaskTracker(), 
+                    taskStatus.getTaskId());
+          }
+        }
+      }
+      for (TaskInProgress tip : job.getReduceTasks()) {
+        for (TaskStatus taskStatus : tip.getTaskStatuses()) {
+          if (taskStatus.getRunState() != TaskStatus.State.RUNNING) {
+            markCompletedTaskAttempt(taskStatus.getTaskTracker(), 
+                    taskStatus.getTaskId());
+          }
+        }
+      }
+    }
+    
+    /**
+     * Remove all 'marked' tasks running on a given {@link TaskTracker}
+     * from the {@link JobTracker}'s data-structures.
+     * This function assumes that the JobTracker is locked on entry.
+     * 
+     * @param taskTracker tasktracker whose 'non-running' tasks are to be purged
+     */
+    private void removeMarkedTasks(String taskTracker) {
+      // Purge all the 'marked' tasks which were running at taskTracker
+      TreeSet<String> markedTaskSet = 
+        (TreeSet<String>) trackerToMarkedTasksMap.get(taskTracker);
+      if (markedTaskSet != null) {
+        for (String taskid : markedTaskSet) {
+          removeTaskEntry(taskid);
+          LOG.info("Removed completed task '" + taskid + "' from '" + 
+                  taskTracker + "'");
+        }
+        // Clear 
+        trackerToMarkedTasksMap.remove(taskTracker);
+      }
+    }
+    
+    /**
+     * Call {@link #removeTaskEntry(String)} for each of the
+     * job's tasks.
+     * When the JobTracker is retiring the long-completed
+     * job, either because it has outlived {@link #RETIRE_JOB_INTERVAL}
+     * or the limit of {@link #MAX_COMPLETE_USER_JOBS_IN_MEMORY} jobs 
+     * has been reached, we can afford to nuke all it's tasks; a little
+     * unsafe, but practically feasible. 
+     * 
+     * @param job the job about to be 'retired'
+     */
+    synchronized private void removeJobTasks(JobInProgress job) { 
+      for (TaskInProgress tip : job.getMapTasks()) {
+        for (TaskStatus taskStatus : tip.getTaskStatuses()) {
+          removeTaskEntry(taskStatus.getTaskId());
+        }
+      }
+      for (TaskInProgress tip : job.getReduceTasks()) {
+        for (TaskStatus taskStatus : tip.getTaskStatuses()) {
+          removeTaskEntry(taskStatus.getTaskId());
+        }
+      }
+    }
+    
+    /**
+     * Safe clean-up all data structures at the end of the 
+     * job (success/failure/killed).
+     * Here we also ensure that for a given user we maintain 
+     * information for only MAX_COMPLETE_USER_JOBS_IN_MEMORY jobs 
+     * on the JobTracker.
+     *  
+     * @param job completed job.
+     */
+    synchronized void finalizeJob(JobInProgress job) {
+      // Mark the 'non-running' tasks for pruning
+      markCompletedJob(job);
+      
+      // Purge oldest jobs and keep at-most MAX_COMPLETE_USER_JOBS_IN_MEMORY jobs of a given user
+      // in memory; information about the purged jobs is available via
+      // JobHistory.
+      synchronized (jobs) {
+        synchronized (jobsByArrival) {
+          synchronized (jobInitQueue) {
+            String jobUser = job.getProfile().getUser();
+            synchronized (userToJobsMap) {
+              ArrayList<JobInProgress> userJobs = 
+                userToJobsMap.get(jobUser);
+              synchronized (userJobs) {
+                while (userJobs.size() > 
+                MAX_COMPLETE_USER_JOBS_IN_MEMORY) {
+                  JobInProgress rjob = userJobs.get(0);
+                  
+                  // Do not delete 'current'
+                  // finished job just yet.
+                  if (rjob == job) {
+                    break;
+                  }
+                  
+                  // Cleanup all datastructures
+                  int rjobRunState = 
+                    rjob.getStatus().getRunState();
+                  if (rjobRunState == JobStatus.SUCCEEDED || 
+                          rjobRunState == JobStatus.FAILED) {
+                    // Ok, this call to removeTaskEntries
+                    // is dangerous is some very very obscure
+                    // cases; e.g. when rjob completed, hit
+                    // MAX_COMPLETE_USER_JOBS_IN_MEMORY job
+                    // limit and yet some task (taskid)
+                    // wasn't complete!
+                    removeJobTasks(rjob);
+                    
+                    userJobs.remove(0);
+                    jobs.remove(rjob.getProfile().getJobId());
+                    jobInitQueue.remove(rjob);
+                    jobsByArrival.remove(rjob);
+                    
+                    LOG.info("Retired job with id: '" + 
+                            rjob.getProfile().getJobId() + "'");
+                  } else {
+                    // Do not remove jobs that aren't complete.
+                    // Stop here, and let the next pass take
+                    // care of purging jobs.
+                    break;
+                  }
+                }
+              }
+            }
+          }
+        }
+      }
     }
 
     ///////////////////////////////////////////////////////
@@ -736,26 +937,46 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     public synchronized HeartbeatResponse heartbeat(TaskTrackerStatus status, 
             boolean initialContact, boolean acceptNewTasks, short responseId) 
     throws IOException {
-      LOG.debug("Got heartbeat from: " + status.getTrackerName() + 
+        LOG.debug("Got heartbeat from: " + status.getTrackerName() + 
               " (initialContact: " + initialContact + 
               " acceptNewTasks: " + acceptNewTasks + ")" +
               " with responseId: " + responseId);
       
         // First check if the last heartbeat response got through 
         String trackerName = status.getTrackerName();
-        Short oldResponseId = trackerToHeartbeatResponseIDMap.get(trackerName);
-      
-        short newResponseId = (short)(responseId + 1);
-        if (!initialContact && oldResponseId != null && 
-                oldResponseId.shortValue() != responseId) {
-            newResponseId = oldResponseId.shortValue();
+        HeartbeatResponse prevHeartbeatResponse =
+            trackerToHeartbeatResponseMap.get(trackerName);
+
+        if (initialContact != true) {
+            // If this isn't the 'initial contact' from the tasktracker,
+            // there is something seriously wrong if the JobTracker has
+            // no record of the 'previous heartbeat'; if so, ask the 
+            // tasktracker to re-initialize itself.
+            if (prevHeartbeatResponse == null) {
+                LOG.warn("Serious problem, cannot find record of 'previous' " +
+                    "heartbeat for '" + trackerName + 
+                    "'; reinitializing the tasktracker");
+                return new HeartbeatResponse(responseId, 
+                        new TaskTrackerAction[] {new ReinitTrackerAction()});
+
+            }
+                
+            // It is completely safe to ignore a 'duplicate' from a tracker
+            // since we are guaranteed that the tracker sends the same 
+            // 'heartbeat' when rpcs are lost. 
+            // {@see TaskTracker.transmitHeartbeat()}
+            if (prevHeartbeatResponse.getResponseId() != responseId) {
+                LOG.info("Ignoring 'duplicate' heartbeat from '" + 
+                        trackerName + "'");
+                return prevHeartbeatResponse;
+            }
         }
       
         // Process this heartbeat 
-        if (!processHeartbeat(status, initialContact, 
-                (newResponseId != responseId))) {
-            if (oldResponseId != null) {
-                trackerToHeartbeatResponseIDMap.remove(trackerName);
+        short newResponseId = (short)(responseId + 1);
+        if (!processHeartbeat(status, initialContact)) {
+            if (prevHeartbeatResponse != null) {
+                trackerToHeartbeatResponseMap.remove(trackerName);
             }
 
             return new HeartbeatResponse(newResponseId, 
@@ -784,12 +1005,12 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         response.setActions(
                 actions.toArray(new TaskTrackerAction[actions.size()]));
         
-        // Update the trackerToHeartbeatResponseIDMap
-        if (newResponseId != responseId) {
-            trackerToHeartbeatResponseIDMap.put(trackerName, 
-                    new Short(newResponseId));
-        }
+        // Update the trackerToHeartbeatResponseMap
+        trackerToHeartbeatResponseMap.put(trackerName, response);
 
+        // Done processing the hearbeat, now remove 'marked' tasks
+        removeMarkedTasks(trackerName);
+        
         return response;
     }
     
@@ -824,12 +1045,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
      * Process incoming heartbeat messages from the task trackers.
      */
     private synchronized boolean processHeartbeat(
-            TaskTrackerStatus trackerStatus, 
-            boolean initialContact, boolean updateStatusTimestamp) {
+            TaskTrackerStatus trackerStatus, boolean initialContact) {
         String trackerName = trackerStatus.getTrackerName();
-        if (initialContact || updateStatusTimestamp) {
-          trackerStatus.setLastSeen(System.currentTimeMillis());
-        }
+        trackerStatus.setLastSeen(System.currentTimeMillis());
 
         synchronized (taskTrackers) {
             synchronized (trackerExpiryQueue) {
@@ -857,7 +1075,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         }
 
         updateTaskStatuses(trackerStatus);
-        //LOG.info("Got heartbeat from "+trackerName);
+
         return true;
     }
 
@@ -1028,7 +1246,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                         killList.add(new KillTaskAction(killTaskId));
                         LOG.debug(taskTracker + " -> KillTaskAction: " + killTaskId);
                     } else {
-                      //killTasksList.add(new KillJobAction(taskId));
                         String killJobId = tip.getJob().getStatus().getJobId(); 
                         killJobIds.add(killJobId);
                     }
@@ -1051,14 +1268,28 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
      * map task outputs.
      */
     public synchronized MapOutputLocation[] 
-             locateMapOutputs(String jobId, int[] mapTasksNeeded, int reduce) {
-        ArrayList result = new ArrayList(mapTasksNeeded.length);
+             locateMapOutputs(String jobId, int[] mapTasksNeeded, int reduce) 
+    throws IOException {
+        // Check to make sure that the job hasn't 'completed'.
         JobInProgress job = getJob(jobId);
+        if (job.status.getRunState() != JobStatus.RUNNING) {
+          return new MapOutputLocation[0];
+        }
+        
+        ArrayList result = new ArrayList(mapTasksNeeded.length);
         for (int i = 0; i < mapTasksNeeded.length; i++) {
           TaskStatus status = job.findFinishedMap(mapTasksNeeded[i]);
           if (status != null) {
              String trackerId = 
                (String) taskidToTrackerMap.get(status.getTaskId());
+             // Safety check, if we can't find the taskid in 
+             // taskidToTrackerMap and job isn't 'running', then just
+             // return an empty array
+             if (trackerId == null && 
+                     job.status.getRunState() != JobStatus.RUNNING) {
+               return new MapOutputLocation[0];
+             }
+             
              TaskTrackerStatus tracker;
              synchronized (taskTrackers) {
                tracker = (TaskTrackerStatus) taskTrackers.get(trackerId);
@@ -1108,10 +1339,22 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         synchronized (jobs) {
             synchronized (jobsByArrival) {
                 synchronized (jobInitQueue) {
-                    jobs.put(job.getProfile().getJobId(), job);
-                    jobsByArrival.add(job);
-                    jobInitQueue.add(job);
-                    jobInitQueue.notifyAll();
+                    synchronized (userToJobsMap) {
+                        jobs.put(job.getProfile().getJobId(), job);
+                        String jobUser = job.getProfile().getUser();
+                        if (!userToJobsMap.containsKey(jobUser)) {
+                            userToJobsMap.put(jobUser, 
+                                    new ArrayList<JobInProgress>());
+                        }
+                        ArrayList<JobInProgress> userJobs = 
+                            userToJobsMap.get(jobUser);
+                        synchronized (userJobs) {
+                            userJobs.add(job);
+                        }
+                        jobsByArrival.add(job);
+                        jobInitQueue.add(job);
+                        jobInitQueue.notifyAll();
+                    }
                 }
             }
         }
@@ -1271,8 +1514,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
      * jobs that might be affected.
      */
     void updateTaskStatuses(TaskTrackerStatus status) {
-        for (Iterator it = status.taskReports(); it.hasNext(); ) {
-            TaskStatus report = (TaskStatus) it.next();
+        for (TaskStatus report : status.getTaskReports()) {
             report.setTaskTracker(status.getTrackerName());
             String taskId = report.getTaskId();
             TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(taskId);
@@ -1310,8 +1552,16 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                                    TaskStatus.Phase.MAP, hostname, trackerName, 
                                    myMetrics);
                   }
+                } else if (!tip.isMapTask() && tip.isComplete()) {
+                  // Completed 'reduce' task, not failed;
+                  // only removed from data-structures.
+                  markCompletedTaskAttempt(trackerName, taskId);
                 }
             }
+            
+            // Purge 'marked' tasks, needs to be done  
+            // here to prevent hanging references!
+            removeMarkedTasks(trackerName);
         }
     }
 

+ 8 - 10
src/java/org/apache/hadoop/mapred/MapTask.java

@@ -202,7 +202,7 @@ class MapTask extends Task {
     //spawn a thread to give merge progress heartbeats
     Thread sortProgress = new Thread() {
       public void run() {
-        LOG.info("Started thread: " + getName());
+        LOG.debug("Started thread: " + getName());
         while (true) {
           try {
             reportProgress(umbilical);
@@ -467,26 +467,24 @@ class MapTask extends Task {
       {
         Path [] filename = new Path[numSpills];
         Path [] indexFileName = new Path[numSpills];
-        FSDataInputStream in[] = new FSDataInputStream[numSpills];
-        FSDataInputStream indexIn[] = new FSDataInputStream[numSpills];
         
         for(int i = 0; i < numSpills; i++) {
           filename[i] = mapOutputFile.getSpillFile(getTaskId(), i);
-          in[i] = localFs.open(filename[i]);
           indexFileName[i] = mapOutputFile.getSpillIndexFile(getTaskId(), i);
-          indexIn[i] = localFs.open(indexFileName[i]);
         }
         
         //create a sorter object as we need access to the SegmentDescriptor
         //class and merge methods
         Sorter sorter = new Sorter(localFs, keyClass, valClass, job);
-        sorter.setFactor(numSpills);
         
         for (int parts = 0; parts < partitions; parts++){
           List<SegmentDescriptor> segmentList = new ArrayList(numSpills);
           for(int i = 0; i < numSpills; i++) {
-            long segmentOffset = indexIn[i].readLong();
-            long segmentLength = indexIn[i].readLong();
+            FSDataInputStream indexIn = localFs.open(indexFileName[i]);
+            indexIn.seek(parts * 16);
+            long segmentOffset = indexIn.readLong();
+            long segmentLength = indexIn.readLong();
+            indexIn.close();
             SegmentDescriptor s = sorter.new SegmentDescriptor(segmentOffset,
                 segmentLength, filename[i]);
             s.preserveInput(true);
@@ -513,8 +511,8 @@ class MapTask extends Task {
         finalIndexOut.close();
         //cleanup
         for(int i = 0; i < numSpills; i++) {
-          in[i].close(); localFs.delete(filename[i]);
-          indexIn[i].close(); localFs.delete(indexFileName[i]);
+          localFs.delete(filename[i]);
+          localFs.delete(indexFileName[i]);
         }
       }
     }

+ 61 - 14
src/java/org/apache/hadoop/mapred/TaskInProgress.java

@@ -57,7 +57,6 @@ class TaskInProgress {
     private int partition;
     private JobTracker jobtracker;
     private String id;
-    private String totalTaskIds[];
     private JobInProgress job;
 
     // Status of the TIP
@@ -70,7 +69,13 @@ class TaskInProgress {
     private int completes = 0;
     private boolean failed = false;
     private boolean killed = false;
-    private TreeSet usableTaskIds = new TreeSet();
+
+    // The 'unique' prefix for taskids of this tip
+    String taskIdPrefix;
+    
+    // The 'next' usable taskid of this tip
+    int nextTaskId = 0;
+    
     // Map from task Id -> TaskTracker Id, contains tasks that are
     // currently runnings
     private TreeMap<String, String> activeTasks = new TreeMap();
@@ -139,13 +144,8 @@ class TaskInProgress {
     void init(String jobUniqueString) {
         this.startTime = System.currentTimeMillis();
         this.runSpeculative = conf.getSpeculativeExecution();
-        String uniqueString = makeUniqueString(jobUniqueString);
-        this.id = "tip_" + uniqueString;
-        this.totalTaskIds = new String[MAX_TASK_EXECS + MAX_TASK_FAILURES];
-        for (int i = 0; i < totalTaskIds.length; i++) {
-          totalTaskIds[i] = "task_" + uniqueString + "_" + i;
-          usableTaskIds.add(totalTaskIds[i]);
-        }
+        this.taskIdPrefix = makeUniqueString(jobUniqueString);
+        this.id = "tip_" + this.taskIdPrefix;
     }
 
     ////////////////////////////////////
@@ -180,11 +180,19 @@ class TaskInProgress {
     }
     
     /**
+     * Is this tip complete?
+     * 
+     * @return <code>true</code> if the tip is complete, else <code>false</code>
      */
     public boolean isComplete() {
         return (completes > 0);
     }
+
     /**
+     * Is the given taskid in this tip complete?
+     * 
+     * @param taskid taskid of attempt to check for completion
+     * @return <code>true</code> if taskid is complete, else <code>false</code>
      */
     public boolean isComplete(String taskid) {
         TaskStatus status = (TaskStatus) taskStatuses.get(taskid);
@@ -194,7 +202,11 @@ class TaskInProgress {
         return ((completes > 0) && 
                 (status.getRunState() == TaskStatus.State.SUCCEEDED));
     }
+
     /**
+     * Is the tip a failure?
+     * 
+     * @return <code>true</code> if tip has failed, else <code>false</code>
      */
     public boolean isFailed() {
         return failed;
@@ -293,6 +305,17 @@ class TaskInProgress {
           TaskStatus.State oldState = oldStatus.getRunState();
           TaskStatus.State newState = status.getRunState();
           
+          // We should never recieve a duplicate success/failure/killed
+          // status update for the same taskid! This is a safety check, 
+          // and is addressed better at the TaskTracker to ensure this.
+          // @see {@link TaskTracker.transmitHeartbeat()}
+          if ((newState != TaskStatus.State.RUNNING) && 
+                  (oldState == newState)) {
+              LOG.warn("Recieved duplicate status update of '" + newState + 
+                      "' for '" + taskid + "' of TIP '" + getTIPId() + "'");
+              return false;
+          }
+
           // The task is not allowed to move from completed back to running.
           // We have seen out of order status messagesmoving tasks from complete
           // to running. This is a spot fix, but it should be addressed more
@@ -346,14 +369,29 @@ class TaskInProgress {
 
     /**
      * Indicate that one of the taskids in this TaskInProgress
-     * has successfully completed!
+     * has successfully completed. 
+     * 
+     * However this may not be the first subtask in this 
+     * TaskInProgress to be completed and hence we might not want to 
+     * manipulate the TaskInProgress to note that it is 'complete' just-as-yet.
      */
-    public void completed(String taskid) {
+    void completedTask(String taskid) {
         LOG.info("Task '" + taskid + "' has completed.");
         TaskStatus status = (TaskStatus) taskStatuses.get(taskid);
         status.setRunState(TaskStatus.State.SUCCEEDED);
         activeTasks.remove(taskid);
-
+    }
+    
+    /**
+     * Indicate that one of the taskids in this TaskInProgress
+     * has successfully completed!
+     */
+    public void completed(String taskid) {
+        //
+        // Record that this taskid is complete
+        //
+        completedTask(taskid);
+        
         //
         // Now that the TIP is complete, the other speculative 
         // subtasks will be closed when the owning tasktracker 
@@ -470,8 +508,17 @@ class TaskInProgress {
           execStartTime = System.currentTimeMillis();
         }
 
-        String taskid = (String) usableTaskIds.first();
-        usableTaskIds.remove(taskid);
+        // Create the 'taskid'
+        String taskid = null;
+        if (nextTaskId < (MAX_TASK_EXECS + MAX_TASK_FAILURES)) {
+          taskid = new String("task_" + taskIdPrefix + "_" + nextTaskId);
+          ++nextTaskId;
+        } else {
+          LOG.warn("Exceeded limit of " + (MAX_TASK_EXECS + MAX_TASK_FAILURES) + 
+                  " attempts for the tip '" + getTIPId() + "'");
+          return null;
+        }
+        
         String jobId = job.getProfile().getJobId();
 
         if (isMapTask()) {

+ 7 - 6
src/java/org/apache/hadoop/mapred/TaskRunner.java

@@ -204,17 +204,18 @@ abstract class TaskRunner extends Thread {
       // Add classpath.
       vargs.add("-classpath");
       vargs.add(classPath.toString());
-      // Add main class and its arguments 
-      vargs.add(TaskTracker.Child.class.getName());  // main of Child
-      vargs.add(tracker.taskReportPort + "");        // pass umbilical port
-      vargs.add(t.getTaskId());                      // pass task identifier
-      
+
       // Add java.library.path; necessary for native-hadoop libraries
       String libraryPath = System.getProperty("java.library.path");
       if (libraryPath != null) {
           vargs.add("-Djava.library.path=" + libraryPath);
       }
-      
+
+      // Add main class and its arguments 
+      vargs.add(TaskTracker.Child.class.getName());  // main of Child
+      vargs.add(tracker.taskReportPort + "");        // pass umbilical port
+      vargs.add(t.getTaskId());                      // pass task identifier
+
       // Run java
       runChild((String[])vargs.toArray(new String[0]), workDir);
     } catch (FSError e) {

+ 38 - 12
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -74,6 +74,16 @@ public class TaskTracker
     // last heartbeat response recieved
     short heartbeatResponseId = -1;
 
+    /*
+     * This is the last 'status' report sent by this tracker to the JobTracker.
+     * 
+     * If the rpc call succeeds, this 'status' is cleared-out by this tracker;
+     * indicating that a 'fresh' status report be generated; in the event the
+     * rpc calls fails for whatever reason, the previous status report is sent
+     * again.
+     */
+    TaskTrackerStatus status = null;
+    
     StatusHttpServer server = null;
     
     boolean shuttingDown = false;
@@ -249,6 +259,7 @@ public class TaskTracker
         this.mapTotal = 0;
         this.reduceTotal = 0;
         this.acceptNewTasks = true;
+        this.status = null;
         
         this.minSpaceStart = this.fConf.getLong("mapred.local.dir.minspacestart", 0L);
         this.minSpaceKill = this.fConf.getLong("mapred.local.dir.minspacekill", 0L);
@@ -535,20 +546,27 @@ public class TaskTracker
      * @throws IOException
      */
     private HeartbeatResponse transmitHeartBeat() throws IOException {
+      // 
+      // Check if the last heartbeat got through... 
+      // if so then build the heartbeat information for the JobTracker;
+      // else resend the previous status information.
       //
-      // Build the heartbeat information for the JobTracker
-      //
-      List<TaskStatus> taskReports = 
-        new ArrayList<TaskStatus>(runningTasks.size());
-      synchronized (this) {
-        for (TaskInProgress tip: runningTasks.values()) {
-          taskReports.add(tip.createStatus());
+      if (status == null) {
+        List<TaskStatus> taskReports = 
+          new ArrayList<TaskStatus>(runningTasks.size());
+        synchronized (this) {
+          for (TaskInProgress tip: runningTasks.values()) {
+            taskReports.add(tip.createStatus());
+          }
         }
+        status = 
+          new TaskTrackerStatus(taskTrackerName, localHostname, 
+                  httpPort, taskReports, 
+                  failures); 
+      } else {
+        LOG.info("Resending 'status' to '" + jobTrackAddr.getHostName() +
+                "' with reponseId '" + heartbeatResponseId);
       }
-      TaskTrackerStatus status = 
-        new TaskTrackerStatus(taskTrackerName, localHostname, 
-                httpPort, taskReports, 
-                failures); 
       
       //
       // Check if we should ask for a new Task
@@ -569,10 +587,14 @@ public class TaskTracker
       HeartbeatResponse heartbeatResponse = jobClient.heartbeat(status, 
               justStarted, askForNewTask, 
               heartbeatResponseId);
+      
+      //
+      // The heartbeat got through successfully!
+      //
       heartbeatResponseId = heartbeatResponse.getResponseId();
       
       synchronized (this) {
-        for (TaskStatus taskStatus : taskReports) {
+        for (TaskStatus taskStatus : status.getTaskReports()) {
           if (taskStatus.getRunState() != TaskStatus.State.RUNNING) {
             if (taskStatus.getIsMap()) {
               mapTotal--;
@@ -584,6 +606,10 @@ public class TaskTracker
           }
         }
       }
+
+      // Force a rebuild of 'status' on the next iteration
+      status = null;                                
+
       return heartbeatResponse;
     }
 

+ 13 - 0
src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java

@@ -98,11 +98,24 @@ class TaskTrackerStatus implements Writable {
      * All current tasks at the TaskTracker.  
      *
      * Tasks are tracked by a TaskStatus object.
+     * 
+     * @deprecated use {@link #getTaskReports()} instead
      */
     public Iterator taskReports() {
         return taskReports.iterator();
     }
 
+    /**
+     * Get the current tasks at the TaskTracker.
+     * Tasks are tracked by a {@link TaskStatus} object.
+     * 
+     * @return a list of {@link TaskStatus} representing 
+     *         the current tasks at the TaskTracker.
+     */
+    public List<TaskStatus> getTaskReports() {
+      return taskReports;
+    }
+    
     /**
      * Return the current MapTask count
      */

+ 1 - 0
src/java/org/apache/hadoop/util/NativeCodeLoader.java

@@ -44,6 +44,7 @@ public class NativeCodeLoader {
     } catch (Throwable t) {
       // Ignore failure to load
       LOG.debug("Failed to load native-hadoop with error: " + t);
+      LOG.debug("java.library.path=" + System.getProperty("java.library.path"));
     }
     
     if (!nativeCodeLoaded) {

+ 11 - 7
src/java/org/apache/hadoop/util/RunJar.java

@@ -106,14 +106,18 @@ public class RunJar {
     }
     mainClassName = mainClassName.replaceAll("/", ".");
 
-    final File workDir = File.createTempFile("hadoop-unjar","", 
-        new File( new Configuration().get("hadoop.tmp.dir")) );
+    File tmpDir = new File(new Configuration().get("hadoop.tmp.dir"));
+    tmpDir.mkdirs();
+    if (!tmpDir.isDirectory()) { 
+      System.err.println("Mkdirs failed to create " + tmpDir);
+      System.exit(-1);
+    }
+    final File workDir = File.createTempFile("hadoop-unjar", "", tmpDir );
     workDir.delete();
-    if (!workDir.mkdirs()) {
-      if (!workDir.isDirectory()) {
-        System.err.println("Mkdirs failed to create " + workDir.toString());
-        System.exit(-1);
-      }
+    workDir.mkdirs();
+    if (!workDir.isDirectory()) {
+      System.err.println("Mkdirs failed to create " + workDir);
+      System.exit(-1);
     }
 
     Runtime.getRuntime().addShutdownHook(new Thread() {

+ 1 - 1
src/native/Makefile.am

@@ -36,7 +36,7 @@
 export PLATFORM = $(shell echo $$OS_NAME | tr [A-Z] [a-z])
 
 # List the sub-directories here
-SUBDIRS = src/org/apache/hadoop/io/compress/zlib lib
+SUBDIRS = src/org/apache/hadoop/io/compress/zlib src/org/apache/hadoop/io/compress/lzo lib
 
 # The following export is needed to build libhadoop.so in the 'lib' directory
 export SUBDIRS

+ 1 - 1
src/native/Makefile.in

@@ -207,7 +207,7 @@ sysconfdir = @sysconfdir@
 target_alias = @target_alias@
 
 # List the sub-directories here
-SUBDIRS = src/org/apache/hadoop/io/compress/zlib lib
+SUBDIRS = src/org/apache/hadoop/io/compress/zlib src/org/apache/hadoop/io/compress/lzo lib
 all: config.h
 	$(MAKE) $(AM_MAKEFLAGS) all-recursive
 

+ 2 - 0
src/native/NEWS

@@ -1,3 +1,5 @@
 2006-10-05 Arun C Murthy <arunc@yahoo-inc.com>
   * Initial version of libhadoop released
 
+2007-01-03 Arun C Murthy <arunc@yahoo-inc.com>
+  * Added support for lzo compression library 

+ 36 - 0
src/native/config.h.in

@@ -1,5 +1,8 @@
 /* config.h.in.  Generated from configure.ac by autoheader.  */
 
+/* The 'actual' dynamic-library for '-llzo2' */
+#undef HADOOP_LZO_LIBRARY
+
 /* The 'actual' dynamic-library for '-lz' */
 #undef HADOOP_ZLIB_LIBRARY
 
@@ -18,9 +21,42 @@
 /* Define to 1 if you have the `jvm' library (-ljvm). */
 #undef HAVE_LIBJVM
 
+/* Define to 1 if you have the `lzo2' library (-llzo2). */
+#undef HAVE_LIBLZO2
+
 /* Define to 1 if you have the `z' library (-lz). */
 #undef HAVE_LIBZ
 
+/* Define to 1 if you have the <lzo/lzo1a.h> header file. */
+#undef HAVE_LZO_LZO1A_H
+
+/* Define to 1 if you have the <lzo/lzo1b.h> header file. */
+#undef HAVE_LZO_LZO1B_H
+
+/* Define to 1 if you have the <lzo/lzo1c.h> header file. */
+#undef HAVE_LZO_LZO1C_H
+
+/* Define to 1 if you have the <lzo/lzo1f.h> header file. */
+#undef HAVE_LZO_LZO1F_H
+
+/* Define to 1 if you have the <lzo/lzo1x.h> header file. */
+#undef HAVE_LZO_LZO1X_H
+
+/* Define to 1 if you have the <lzo/lzo1y.h> header file. */
+#undef HAVE_LZO_LZO1Y_H
+
+/* Define to 1 if you have the <lzo/lzo1z.h> header file. */
+#undef HAVE_LZO_LZO1Z_H
+
+/* Define to 1 if you have the <lzo/lzo1.h> header file. */
+#undef HAVE_LZO_LZO1_H
+
+/* Define to 1 if you have the <lzo/lzo2a.h> header file. */
+#undef HAVE_LZO_LZO2A_H
+
+/* Define to 1 if you have the <lzo/lzo_asm.h> header file. */
+#undef HAVE_LZO_LZO_ASM_H
+
 /* Define to 1 if you have the <memory.h> header file. */
 #undef HAVE_MEMORY_H
 

Різницю між файлами не показано, бо вона завелика
+ 604 - 157
src/native/configure


+ 7 - 0
src/native/configure.ac

@@ -64,6 +64,9 @@ AC_SUBST([JNI_LDFLAGS])
 dnl Check for '-lz'
 AC_CHECK_LIB([z], [deflate])
 
+dnl Check for '-llzo2'
+AC_CHECK_LIB([lzo2], [lzo_init])
+
 # Checks for header files.
 dnl Check for Ansi C headers
 AC_HEADER_STDC
@@ -89,6 +92,9 @@ AC_SUBST([JNI_CPPFLAGS])
 dnl Check for zlib headers
 AC_CHECK_HEADERS([zlib.h zconf.h], AC_COMPUTE_NEEDED_DSO(z,HADOOP_ZLIB_LIBRARY), AC_MSG_ERROR(Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package.))
 
+dnl Check for lzo headers
+AC_CHECK_HEADERS([lzo/lzo1.h lzo/lzo1a.h lzo/lzo1b.h lzo/lzo1c.h lzo/lzo1f.h lzo/lzo1x.h lzo/lzo1y.h lzo/lzo1z.h lzo/lzo2a.h lzo/lzo_asm.h], AC_COMPUTE_NEEDED_DSO(lzo2,HADOOP_LZO_LIBRARY), AC_MSG_ERROR(lzo headers were not found... native-hadoop library needs lzo to build. Please install the requisite lzo development package.))
+
 # Checks for typedefs, structures, and compiler characteristics.
 AC_C_CONST
 
@@ -97,6 +103,7 @@ AC_CHECK_FUNCS([memset])
 
 AC_CONFIG_FILES([Makefile
                  src/org/apache/hadoop/io/compress/zlib/Makefile
+                 src/org/apache/hadoop/io/compress/lzo/Makefile
                  lib/Makefile])
 AC_OUTPUT
 

+ 253 - 0
src/native/src/org/apache/hadoop/io/compress/lzo/LzoCompressor.c

@@ -0,0 +1,253 @@
+#if defined HAVE_CONFIG_H
+  #include <config.h>
+#endif
+
+#if defined HAVE_STDIO_H
+  #include <stdio.h>
+#else
+  #error 'stdio.h not found'
+#endif  
+
+#if defined HAVE_STDLIB_H
+  #include <stdlib.h>
+#else
+  #error 'stdlib.h not found'
+#endif  
+
+#include "org_apache_hadoop_io_compress_lzo.h"
+
+// The lzo2 library-handle
+static void *liblzo2 = NULL;
+
+// The lzo 'compressors'
+typedef struct {
+  const char *function;           // The compression function
+  int wrkmem;                     // The 'working memory' needed
+  int compression_level;          // Compression level if required;
+                                  // else UNDEFINED_COMPRESSION_LEVEL
+} lzo_compressor;
+
+#define UNDEFINED_COMPRESSION_LEVEL -999
+
+static lzo_compressor lzo_compressors[] = {
+  /** lzo1 compressors */
+  /* 0 */   {"lzo1_compress", LZO1_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 1 */   {"lzo1_99_compress", LZO1_99_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+
+  /** lzo1a compressors */
+  /* 2 */   {"lzo1a_compress", LZO1A_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 3 */   {"lzo1a_99_compress", LZO1A_99_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+
+  /** lzo1b compressors */
+  /* 4 */   {"lzo1b_compress", LZO1B_MEM_COMPRESS, LZO1B_DEFAULT_COMPRESSION}, 
+  /* 5 */   {"lzo1b_compress", LZO1B_MEM_COMPRESS, LZO1B_BEST_SPEED}, 
+  /* 6 */   {"lzo1b_compress", LZO1B_MEM_COMPRESS, LZO1B_BEST_COMPRESSION}, 
+  /* 7 */   {"lzo1b_1_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 8 */   {"lzo1b_2_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 9 */   {"lzo1b_3_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 10 */  {"lzo1b_4_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 11 */  {"lzo1b_5_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 12 */  {"lzo1b_6_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 13 */  {"lzo1b_7_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 14 */  {"lzo1b_8_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 15 */  {"lzo1b_9_compress", LZO1B_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 16 */  {"lzo1b_99_compress", LZO1B_99_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 17 */  {"lzo1b_999_compress", LZO1B_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  
+  /** lzo1c compressors */
+  /* 18 */  {"lzo1c_compress", LZO1C_MEM_COMPRESS, LZO1C_DEFAULT_COMPRESSION}, 
+  /* 19 */  {"lzo1c_compress", LZO1C_MEM_COMPRESS, LZO1C_BEST_SPEED}, 
+  /* 20 */  {"lzo1c_compress", LZO1C_MEM_COMPRESS, LZO1C_BEST_COMPRESSION}, 
+  /* 21 */  {"lzo1c_1_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 22 */  {"lzo1c_2_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 23 */  {"lzo1c_3_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 24 */  {"lzo1c_4_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 25 */  {"lzo1c_5_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 26 */  {"lzo1c_6_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 27 */  {"lzo1c_7_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 28 */  {"lzo1c_8_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 29 */  {"lzo1c_9_compress", LZO1C_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 30 */  {"lzo1c_99_compress", LZO1C_99_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  /* 31 */  {"lzo1c_999_compress", LZO1C_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL}, 
+  
+  /** lzo1f compressors */
+  /* 32 */  {"lzo1f_1_compress", LZO1F_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 33 */  {"lzo1f_999_compress", LZO1F_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+
+  /** lzo1x compressors */
+  /* 34 */  {"lzo1x_1_compress", LZO1X_1_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 35 */  {"lzo1x_11_compress", LZO1X_1_11_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 36 */  {"lzo1x_12_compress", LZO1X_1_12_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 37 */  {"lzo1x_15_compress", LZO1X_1_15_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 38 */  {"lzo1x_999_compress", LZO1X_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+
+  /** lzo1y compressors */
+  /* 39 */  {"lzo1y_1_compress", LZO1Y_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+  /* 40 */  {"lzo1y_999_compress", LZO1Y_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+
+  /** lzo1z compressors */
+  /* 41 */  {"lzo1z_999_compress", LZO1Z_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+
+  /** lzo2a compressors */
+  /* 42 */  {"lzo2a_999_compress", LZO2A_999_MEM_COMPRESS, UNDEFINED_COMPRESSION_LEVEL},
+};
+
+// The second lzo* compressor prototype - this really should be in lzoconf.h!
+typedef int
+(__LZO_CDECL *lzo_compress2_t)   ( const lzo_bytep src, lzo_uint  src_len,
+                                  lzo_bytep dst, lzo_uintp dst_len,
+                                  lzo_voidp wrkmem, int compression_level );
+
+static jfieldID LzoCompressor_finish;
+static jfieldID LzoCompressor_finished;
+static jfieldID LzoCompressor_uncompressedDirectBuf;
+static jfieldID LzoCompressor_uncompressedDirectBufLen;
+static jfieldID LzoCompressor_compressedDirectBuf;
+static jfieldID LzoCompressor_directBufferSize;
+static jfieldID LzoCompressor_lzoCompressor;
+static jfieldID LzoCompressor_workingMemoryBufLen;
+static jfieldID LzoCompressor_workingMemoryBuf;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_compress_lzo_LzoCompressor_initIDs(
+	JNIEnv *env, jclass class
+	) {
+	// Load liblzo2.so
+	liblzo2 = dlopen(HADOOP_LZO_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
+	if (!liblzo2) {
+		THROW(env, "java/lang/UnsatisfiedLinkError", "Cannot load liblzo2.so!");
+	  return;
+	}
+    
+  LzoCompressor_finish = (*env)->GetFieldID(env, class, "finish", "Z");
+  LzoCompressor_finished = (*env)->GetFieldID(env, class, "finished", "Z");
+  LzoCompressor_uncompressedDirectBuf = (*env)->GetFieldID(env, class, 
+                                                    "uncompressedDirectBuf", 
+                                                    "Ljava/nio/Buffer;");
+  LzoCompressor_uncompressedDirectBufLen = (*env)->GetFieldID(env, class, 
+                                            "uncompressedDirectBufLen", "I");
+  LzoCompressor_compressedDirectBuf = (*env)->GetFieldID(env, class, 
+                                                        "compressedDirectBuf",
+                                                        "Ljava/nio/Buffer;");
+  LzoCompressor_directBufferSize = (*env)->GetFieldID(env, class, 
+                                            "directBufferSize", "I");
+  LzoCompressor_lzoCompressor = (*env)->GetFieldID(env, class, 
+                                          "lzoCompressor", "J");
+  LzoCompressor_workingMemoryBufLen = (*env)->GetFieldID(env, class,
+                                                "workingMemoryBufLen", "I");
+  LzoCompressor_workingMemoryBuf = (*env)->GetFieldID(env, class, 
+                                              "workingMemoryBuf", 
+                                              "Ljava/nio/Buffer;");
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_compress_lzo_LzoCompressor_init(
+  JNIEnv *env, jobject this, jint compressor 
+  ) {
+  const char *lzo_compressor_function = lzo_compressors[compressor].function;
+ 
+  // Locate the requisite symbols from liblzo2.so
+  dlerror();                                 // Clear any existing error
+
+  // Initialize the lzo library 
+  void *lzo_init_func_ptr = NULL;
+  typedef int (__LZO_CDECL *lzo_init_t) (unsigned,int,int,int,int,int,int,int,int,int);
+  LOAD_DYNAMIC_SYMBOL(lzo_init_func_ptr, env, liblzo2, "__lzo_init_v2");
+  lzo_init_t lzo_init_function = (lzo_init_t)(lzo_init_func_ptr);
+  int rv = lzo_init_function(LZO_VERSION, (int)sizeof(short), (int)sizeof(int), 
+              (int)sizeof(long), (int)sizeof(lzo_uint32), (int)sizeof(lzo_uint), 
+              (int)lzo_sizeof_dict_t, (int)sizeof(char*), (int)sizeof(lzo_voidp),
+              (int)sizeof(lzo_callback_t));
+  if (rv != LZO_E_OK) {
+    THROW(env, "Ljava/lang/InternalError", "Could not initialize lzo library!");
+    return;
+  }
+  
+  // Save the compressor-function into LzoCompressor_lzoCompressor
+  void *compressor_func_ptr = NULL;
+  LOAD_DYNAMIC_SYMBOL(compressor_func_ptr, env, liblzo2, lzo_compressor_function);
+  (*env)->SetLongField(env, this, LzoCompressor_lzoCompressor,
+                       JLONG(compressor_func_ptr));
+  
+  // Save the compressor-function into LzoCompressor_lzoCompressor
+  (*env)->SetIntField(env, this, LzoCompressor_workingMemoryBufLen,
+                      lzo_compressors[compressor].wrkmem);
+
+  return;
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_io_compress_lzo_LzoCompressor_compressBytesDirect(
+  JNIEnv *env, jobject this, jint compressor 
+	) {
+  const char *lzo_compressor_function = lzo_compressors[compressor].function;
+
+	// Get members of LzoCompressor
+	jobject uncompressed_direct_buf = (*env)->GetObjectField(env, this, 
+									                    LzoCompressor_uncompressedDirectBuf);
+	lzo_uint uncompressed_direct_buf_len = (*env)->GetIntField(env, this, 
+									                  LzoCompressor_uncompressedDirectBufLen);
+
+	jobject compressed_direct_buf = (*env)->GetObjectField(env, this, 
+									                        LzoCompressor_compressedDirectBuf);
+	lzo_uint compressed_direct_buf_len = (*env)->GetIntField(env, this, 
+									                            LzoCompressor_directBufferSize);
+
+	jobject working_memory_buf = (*env)->GetObjectField(env, this, 
+									                      LzoCompressor_workingMemoryBuf);
+
+  jlong lzo_compressor_funcptr = (*env)->GetLongField(env, this,
+                  LzoCompressor_lzoCompressor);
+
+  // Get direct buffers
+	lzo_bytep uncompressed_bytes = (*env)->GetDirectBufferAddress(env, 
+                                            uncompressed_direct_buf);
+  if (uncompressed_bytes == 0) {
+    	return (jint)0;
+	}
+	
+	lzo_bytep compressed_bytes = (*env)->GetDirectBufferAddress(env, 
+                                            compressed_direct_buf);
+  if (compressed_bytes == 0) {
+		return (jint)0;
+	}
+	
+  lzo_voidp workmem = (*env)->GetDirectBufferAddress(env, working_memory_buf);
+  if (workmem == 0) {
+    return (jint)0;
+  }
+  
+	// Compress
+  lzo_uint no_compressed_bytes = compressed_direct_buf_len;
+	int rv = 0;
+  int compression_level = lzo_compressors[compressor].compression_level;
+  if (compression_level == UNDEFINED_COMPRESSION_LEVEL) {
+    lzo_compress_t fptr = (lzo_compress_t) FUNC_PTR(lzo_compressor_funcptr);
+    rv = fptr(uncompressed_bytes, uncompressed_direct_buf_len,
+              compressed_bytes, &no_compressed_bytes, 
+              workmem);
+  } else {
+    lzo_compress2_t fptr = (lzo_compress2_t) FUNC_PTR(lzo_compressor_funcptr);
+    rv = fptr(uncompressed_bytes, uncompressed_direct_buf_len,
+              compressed_bytes, &no_compressed_bytes, 
+              workmem, compression_level); 
+  }
+
+  if (rv == LZO_E_OK) {
+    // lzo compresses all input data
+    (*env)->SetIntField(env, this, 
+                LzoCompressor_uncompressedDirectBufLen, 0);
+  } else {
+    const int msg_len = 32;
+    char exception_msg[msg_len];
+    snprintf(exception_msg, msg_len, "%s returned: %d", lzo_compressor_function, rv);
+    THROW(env, "java/lang/InternalError", exception_msg);
+  }
+
+  return (jint)no_compressed_bytes;
+}
+
+/**
+ * vim: sw=2: ts=2: et:
+ */
+

+ 194 - 0
src/native/src/org/apache/hadoop/io/compress/lzo/LzoDecompressor.c

@@ -0,0 +1,194 @@
+#if defined HAVE_CONFIG_H
+  #include <config.h>
+#endif
+
+#if defined HAVE_STDIO_H
+  #include <stdio.h>
+#else
+  #error 'stdio.h not found'
+#endif  
+
+#if defined HAVE_STDLIB_H
+  #include <stdlib.h>
+#else
+  #error 'stdlib.h not found'
+#endif  
+
+#include "org_apache_hadoop_io_compress_lzo.h"
+
+// The lzo2 library-handle
+static void *liblzo2 = NULL;
+
+// The lzo 'decompressors'
+static char* lzo_decompressors[] = {
+  /** lzo1 decompressors */
+  /* 0 */   "lzo1_decompress", 
+  
+  /** lzo1a compressors */
+  /* 1 */   "lzo1a_decompress",
+
+  /** lzo1b compressors */
+  /* 2 */   "lzo1b_decompress", 
+  /* 3 */   "lzo1b_decompress_safe",
+
+  /** lzo1c compressors */
+  /* 4 */   "lzo1c_decompress",
+  /* 5 */   "lzo1c_decompress_safe",
+  /* 6 */   "lzo1c_decompress_asm",
+  /* 7 */   "lzo1c_decompress_asm_safe",
+  
+  /** lzo1f compressors */
+  /* 8 */   "lzo1f_decompress",
+  /* 9 */   "lzo1f_decompress_safe",
+  /* 10 */  "lzo1f_decompress_asm_fast",
+  /* 11 */  "lzo1f_decompress_asm_fast_safe",
+
+  /** lzo1x compressors */
+  /* 12 */  "lzo1x_decompress",
+  /* 13 */  "lzo1x_decompress_safe",
+  /* 14 */  "lzo1x_decompress_asm",
+  /* 15 */  "lzo1x_decompress_asm_safe",
+  /* 16 */  "lzo1x_decompress_asm_fast",
+  /* 17 */  "lzo1x_decompress_asm_fast_safe"
+  
+  /** lzo1y compressors */
+  /* 18 */  "lzo1y_decompress",
+  /* 19 */  "lzo1y_decompress_safe",
+  /* 20 */  "lzo1y_decompress_asm",
+  /* 21 */  "lzo1y_decompress_asm_safe",
+  /* 22 */  "lzo1y_decompress_asm_fast",
+  /* 23 */  "lzo1y_decompress_asm_fast_safe",
+
+  /** lzo1z compressors */
+  /* 24 */  "lzo1z_decompress", 
+  /* 25 */  "lzo1z_decompress_safe",
+
+  /** lzo2a compressors */
+  /* 26 */  "lzo2a_decompress",
+  /* 27 */  "lzo2a_decompress_safe"
+};
+
+static jfieldID LzoDecompressor_finished;
+static jfieldID LzoDecompressor_compressedDirectBuf;
+static jfieldID LzoDecompressor_compressedDirectBufLen;
+static jfieldID LzoDecompressor_uncompressedDirectBuf;
+static jfieldID LzoDecompressor_directBufferSize;
+static jfieldID LzoDecompressor_lzoDecompressor;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_compress_lzo_LzoDecompressor_initIDs(
+	JNIEnv *env, jclass class
+	) {
+	// Load liblzo2.so
+	liblzo2 = dlopen(HADOOP_LZO_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
+	if (!liblzo2) {
+		THROW(env, "java/lang/UnsatisfiedLinkError", "Cannot load liblzo2.so!");
+	  return;
+	}
+    
+  LzoDecompressor_finished = (*env)->GetFieldID(env, class, "finished", "Z");
+  LzoDecompressor_compressedDirectBuf = (*env)->GetFieldID(env, class, 
+                                                "compressedDirectBuf", 
+                                                "Ljava/nio/Buffer;");
+  LzoDecompressor_compressedDirectBufLen = (*env)->GetFieldID(env, class, 
+                                                    "compressedDirectBufLen", "I");
+  LzoDecompressor_uncompressedDirectBuf = (*env)->GetFieldID(env, class, 
+                                                  "uncompressedDirectBuf", 
+                                                  "Ljava/nio/Buffer;");
+  LzoDecompressor_directBufferSize = (*env)->GetFieldID(env, class, 
+                                              "directBufferSize", "I");
+  LzoDecompressor_lzoDecompressor = (*env)->GetFieldID(env, class,
+                                              "lzoDecompressor", "J");
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_compress_lzo_LzoDecompressor_init(
+  JNIEnv *env, jobject this, jint decompressor 
+  ) {
+  const char *lzo_decompressor_function = lzo_decompressors[decompressor];
+ 
+  // Locate the requisite symbols from liblzo2.so
+  dlerror();                                 // Clear any existing error
+
+  // Initialize the lzo library 
+  void *lzo_init_func_ptr = NULL;
+  typedef int (__LZO_CDECL *lzo_init_t) (unsigned,int,int,int,int,int,int,int,int,int);
+  LOAD_DYNAMIC_SYMBOL(lzo_init_func_ptr, env, liblzo2, "__lzo_init_v2");
+  lzo_init_t lzo_init_function = (lzo_init_t)(lzo_init_func_ptr);
+  int rv = lzo_init_function(LZO_VERSION, (int)sizeof(short), (int)sizeof(int), 
+              (int)sizeof(long), (int)sizeof(lzo_uint32), (int)sizeof(lzo_uint), 
+              (int)lzo_sizeof_dict_t, (int)sizeof(char*), (int)sizeof(lzo_voidp),
+              (int)sizeof(lzo_callback_t));
+  if (rv != LZO_E_OK) {
+    THROW(env, "Ljava/lang/InternalError", "Could not initialize lzo library!");
+    return;
+  }
+  
+  // Save the decompressor-function into LzoDecompressor_lzoDecompressor
+  void *decompressor_func_ptr = NULL;
+  LOAD_DYNAMIC_SYMBOL(decompressor_func_ptr, env, liblzo2,
+      lzo_decompressor_function);
+  (*env)->SetLongField(env, this, LzoDecompressor_lzoDecompressor,
+                       JLONG(decompressor_func_ptr));
+
+  return;
+}
+
+JNIEXPORT jint JNICALL
+Java_org_apache_hadoop_io_compress_lzo_LzoDecompressor_decompressBytesDirect(
+	JNIEnv *env, jobject this, jint decompressor
+	) {
+  const char *lzo_decompressor_function = lzo_decompressors[decompressor];
+
+	// Get members of LzoDecompressor
+	jobject compressed_direct_buf = (*env)->GetObjectField(env, this,
+                                              LzoDecompressor_compressedDirectBuf);
+	lzo_uint compressed_direct_buf_len = (*env)->GetIntField(env, this, 
+                        		  							LzoDecompressor_compressedDirectBufLen);
+
+	jobject uncompressed_direct_buf = (*env)->GetObjectField(env, this, 
+                            								  LzoDecompressor_uncompressedDirectBuf);
+	lzo_uint uncompressed_direct_buf_len = (*env)->GetIntField(env, this,
+                                                LzoDecompressor_directBufferSize);
+
+  jlong lzo_decompressor_funcptr = (*env)->GetLongField(env, this,
+                                              LzoDecompressor_lzoDecompressor);
+
+  // Get direct buffers
+	lzo_bytep uncompressed_bytes = (*env)->GetDirectBufferAddress(env, 
+											                    uncompressed_direct_buf);
+ 	if (uncompressed_bytes == 0) {
+    return (jint)0;
+	}
+	
+	lzo_bytep compressed_bytes = (*env)->GetDirectBufferAddress(env, 
+										                    compressed_direct_buf);
+  if (compressed_bytes == 0) {
+		return (jint)0;
+	}
+	
+	// Decompress
+  lzo_uint no_uncompressed_bytes = uncompressed_direct_buf_len;
+  lzo_decompress_t fptr = (lzo_decompress_t) FUNC_PTR(lzo_decompressor_funcptr);
+	int rv = fptr(compressed_bytes, compressed_direct_buf_len,
+                uncompressed_bytes, &no_uncompressed_bytes,
+                NULL); 
+
+  if (rv == LZO_E_OK) {
+    // lzo decompresses all input data
+    (*env)->SetIntField(env, this, LzoDecompressor_compressedDirectBufLen, 0);
+  } else {
+    const int msg_len = 32;
+    char exception_msg[msg_len];
+    snprintf(exception_msg, msg_len, "%s returned: %d", 
+              lzo_decompressor_function, rv);
+    THROW(env, "java/lang/InternalError", exception_msg);
+  }
+  
+  return no_uncompressed_bytes;
+}
+
+/**
+ * vim: sw=2: ts=2: et:
+ */
+

+ 50 - 0
src/native/src/org/apache/hadoop/io/compress/lzo/Makefile.am

@@ -0,0 +1,50 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+#
+# Makefile template for building native 'lzo' for hadoop.
+#
+
+#
+# Notes: 
+# 1. This makefile is designed to do the actual builds in $(HADOOP_HOME)/build/native/${os.name}-${os.arch}/$(subdir) .
+# 2. This makefile depends on the following environment variables to function correctly:
+#    * HADOOP_NATIVE_SRCDIR 
+#    * JAVA_HOME
+#    * JVM_DATA_MODEL
+#    * OS_ARCH 
+#    * PLATFORM
+#    All these are setup by build.xml and/or the top-level makefile.
+# 3. The creation of requisite jni headers/stubs are also done by build.xml and they are
+#    assumed to be in $(HADOOP_HOME)/build/native/src/org/apache/hadoop/io/compress/lzo.
+#
+
+# The 'vpath directive' to locate the actual source files 
+vpath %.c $(HADOOP_NATIVE_SRCDIR)/$(subdir)
+
+AM_CPPFLAGS = @JNI_CPPFLAGS@ -I$(HADOOP_NATIVE_SRCDIR)/src
+AM_LDFLAGS = @JNI_LDFLAGS@
+AM_CFLAGS = -g -Wall -fPIC -O2 -m$(JVM_DATA_MODEL)
+
+noinst_LTLIBRARIES = libnativelzo.la
+libnativelzo_la_SOURCES = LzoCompressor.c LzoDecompressor.c
+libnativelzo_la_LIBADD = -ldl -ljvm
+
+#
+#vim: sw=4: ts=4: noet
+#

+ 469 - 0
src/native/src/org/apache/hadoop/io/compress/lzo/Makefile.in

@@ -0,0 +1,469 @@
+# Makefile.in generated by automake 1.9.6 from Makefile.am.
+# @configure_input@
+
+# Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
+# 2003, 2004, 2005  Free Software Foundation, Inc.
+# This Makefile.in is free software; the Free Software Foundation
+# gives unlimited permission to copy and/or distribute it,
+# with or without modifications, as long as this notice is preserved.
+
+# This program is distributed in the hope that it will be useful,
+# but WITHOUT ANY WARRANTY, to the extent permitted by law; without
+# even the implied warranty of MERCHANTABILITY or FITNESS FOR A
+# PARTICULAR PURPOSE.
+
+@SET_MAKE@
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+#
+# Makefile template for building native 'lzo' for hadoop.
+#
+
+#
+# Notes: 
+# 1. This makefile is designed to do the actual builds in $(HADOOP_HOME)/build/native/${os.name}-${os.arch}/$(subdir) .
+# 2. This makefile depends on the following environment variables to function correctly:
+#    * HADOOP_NATIVE_SRCDIR 
+#    * JAVA_HOME
+#    * JVM_DATA_MODEL
+#    * OS_ARCH 
+#    * PLATFORM
+#    All these are setup by build.xml and/or the top-level makefile.
+# 3. The creation of requisite jni headers/stubs are also done by build.xml and they are
+#    assumed to be in $(HADOOP_HOME)/build/native/src/org/apache/hadoop/io/compress/lzo.
+#
+
+srcdir = @srcdir@
+top_srcdir = @top_srcdir@
+VPATH = @srcdir@
+pkgdatadir = $(datadir)/@PACKAGE@
+pkglibdir = $(libdir)/@PACKAGE@
+pkgincludedir = $(includedir)/@PACKAGE@
+top_builddir = ../../../../../../..
+am__cd = CDPATH="$${ZSH_VERSION+.}$(PATH_SEPARATOR)" && cd
+INSTALL = @INSTALL@
+install_sh_DATA = $(install_sh) -c -m 644
+install_sh_PROGRAM = $(install_sh) -c
+install_sh_SCRIPT = $(install_sh) -c
+INSTALL_HEADER = $(INSTALL_DATA)
+transform = $(program_transform_name)
+NORMAL_INSTALL = :
+PRE_INSTALL = :
+POST_INSTALL = :
+NORMAL_UNINSTALL = :
+PRE_UNINSTALL = :
+POST_UNINSTALL = :
+build_triplet = @build@
+host_triplet = @host@
+subdir = src/org/apache/hadoop/io/compress/lzo
+DIST_COMMON = $(srcdir)/Makefile.am $(srcdir)/Makefile.in
+ACLOCAL_M4 = $(top_srcdir)/aclocal.m4
+am__aclocal_m4_deps = $(top_srcdir)/acinclude.m4 \
+	$(top_srcdir)/configure.ac
+am__configure_deps = $(am__aclocal_m4_deps) $(CONFIGURE_DEPENDENCIES) \
+	$(ACLOCAL_M4)
+mkinstalldirs = $(install_sh) -d
+CONFIG_HEADER = $(top_builddir)/config.h
+CONFIG_CLEAN_FILES =
+LTLIBRARIES = $(noinst_LTLIBRARIES)
+libnativelzo_la_DEPENDENCIES =
+am_libnativelzo_la_OBJECTS = LzoCompressor.lo LzoDecompressor.lo
+libnativelzo_la_OBJECTS = $(am_libnativelzo_la_OBJECTS)
+DEFAULT_INCLUDES = -I. -I$(srcdir) -I$(top_builddir)
+depcomp = $(SHELL) $(top_srcdir)/config/depcomp
+am__depfiles_maybe = depfiles
+COMPILE = $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) \
+	$(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS)
+LTCOMPILE = $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) \
+	$(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) \
+	$(AM_CFLAGS) $(CFLAGS)
+CCLD = $(CC)
+LINK = $(LIBTOOL) --tag=CC --mode=link $(CCLD) $(AM_CFLAGS) $(CFLAGS) \
+	$(AM_LDFLAGS) $(LDFLAGS) -o $@
+SOURCES = $(libnativelzo_la_SOURCES)
+DIST_SOURCES = $(libnativelzo_la_SOURCES)
+ETAGS = etags
+CTAGS = ctags
+DISTFILES = $(DIST_COMMON) $(DIST_SOURCES) $(TEXINFOS) $(EXTRA_DIST)
+ACLOCAL = @ACLOCAL@
+AMDEP_FALSE = @AMDEP_FALSE@
+AMDEP_TRUE = @AMDEP_TRUE@
+AMTAR = @AMTAR@
+AR = @AR@
+AUTOCONF = @AUTOCONF@
+AUTOHEADER = @AUTOHEADER@
+AUTOMAKE = @AUTOMAKE@
+AWK = @AWK@
+CC = @CC@
+CCDEPMODE = @CCDEPMODE@
+CFLAGS = @CFLAGS@
+CPP = @CPP@
+CPPFLAGS = @CPPFLAGS@
+CXX = @CXX@
+CXXCPP = @CXXCPP@
+CXXDEPMODE = @CXXDEPMODE@
+CXXFLAGS = @CXXFLAGS@
+CYGPATH_W = @CYGPATH_W@
+DEFS = @DEFS@
+DEPDIR = @DEPDIR@
+ECHO = @ECHO@
+ECHO_C = @ECHO_C@
+ECHO_N = @ECHO_N@
+ECHO_T = @ECHO_T@
+EGREP = @EGREP@
+EXEEXT = @EXEEXT@
+F77 = @F77@
+FFLAGS = @FFLAGS@
+INSTALL_DATA = @INSTALL_DATA@
+INSTALL_PROGRAM = @INSTALL_PROGRAM@
+INSTALL_SCRIPT = @INSTALL_SCRIPT@
+INSTALL_STRIP_PROGRAM = @INSTALL_STRIP_PROGRAM@
+JNI_CPPFLAGS = @JNI_CPPFLAGS@
+JNI_LDFLAGS = @JNI_LDFLAGS@
+LDFLAGS = @LDFLAGS@
+LIBOBJS = @LIBOBJS@
+LIBS = @LIBS@
+LIBTOOL = @LIBTOOL@
+LN_S = @LN_S@
+LTLIBOBJS = @LTLIBOBJS@
+MAKEINFO = @MAKEINFO@
+OBJEXT = @OBJEXT@
+PACKAGE = @PACKAGE@
+PACKAGE_BUGREPORT = @PACKAGE_BUGREPORT@
+PACKAGE_NAME = @PACKAGE_NAME@
+PACKAGE_STRING = @PACKAGE_STRING@
+PACKAGE_TARNAME = @PACKAGE_TARNAME@
+PACKAGE_VERSION = @PACKAGE_VERSION@
+PATH_SEPARATOR = @PATH_SEPARATOR@
+RANLIB = @RANLIB@
+SET_MAKE = @SET_MAKE@
+SHELL = @SHELL@
+STRIP = @STRIP@
+VERSION = @VERSION@
+ac_ct_AR = @ac_ct_AR@
+ac_ct_CC = @ac_ct_CC@
+ac_ct_CXX = @ac_ct_CXX@
+ac_ct_F77 = @ac_ct_F77@
+ac_ct_RANLIB = @ac_ct_RANLIB@
+ac_ct_STRIP = @ac_ct_STRIP@
+am__fastdepCC_FALSE = @am__fastdepCC_FALSE@
+am__fastdepCC_TRUE = @am__fastdepCC_TRUE@
+am__fastdepCXX_FALSE = @am__fastdepCXX_FALSE@
+am__fastdepCXX_TRUE = @am__fastdepCXX_TRUE@
+am__include = @am__include@
+am__leading_dot = @am__leading_dot@
+am__quote = @am__quote@
+am__tar = @am__tar@
+am__untar = @am__untar@
+bindir = @bindir@
+build = @build@
+build_alias = @build_alias@
+build_cpu = @build_cpu@
+build_os = @build_os@
+build_vendor = @build_vendor@
+datadir = @datadir@
+exec_prefix = @exec_prefix@
+host = @host@
+host_alias = @host_alias@
+host_cpu = @host_cpu@
+host_os = @host_os@
+host_vendor = @host_vendor@
+includedir = @includedir@
+infodir = @infodir@
+install_sh = @install_sh@
+libdir = @libdir@
+libexecdir = @libexecdir@
+localstatedir = @localstatedir@
+mandir = @mandir@
+mkdir_p = @mkdir_p@
+oldincludedir = @oldincludedir@
+prefix = @prefix@
+program_transform_name = @program_transform_name@
+sbindir = @sbindir@
+sharedstatedir = @sharedstatedir@
+sysconfdir = @sysconfdir@
+target_alias = @target_alias@
+AM_CPPFLAGS = @JNI_CPPFLAGS@ -I$(HADOOP_NATIVE_SRCDIR)/src
+AM_LDFLAGS = @JNI_LDFLAGS@
+AM_CFLAGS = -g -Wall -fPIC -O2 -m$(JVM_DATA_MODEL)
+noinst_LTLIBRARIES = libnativelzo.la
+libnativelzo_la_SOURCES = LzoCompressor.c LzoDecompressor.c
+libnativelzo_la_LIBADD = -ldl -ljvm
+all: all-am
+
+.SUFFIXES:
+.SUFFIXES: .c .lo .o .obj
+$(srcdir)/Makefile.in:  $(srcdir)/Makefile.am  $(am__configure_deps)
+	@for dep in $?; do \
+	  case '$(am__configure_deps)' in \
+	    *$$dep*) \
+	      cd $(top_builddir) && $(MAKE) $(AM_MAKEFLAGS) am--refresh \
+		&& exit 0; \
+	      exit 1;; \
+	  esac; \
+	done; \
+	echo ' cd $(top_srcdir) && $(AUTOMAKE) --gnu  src/org/apache/hadoop/io/compress/lzo/Makefile'; \
+	cd $(top_srcdir) && \
+	  $(AUTOMAKE) --gnu  src/org/apache/hadoop/io/compress/lzo/Makefile
+.PRECIOUS: Makefile
+Makefile: $(srcdir)/Makefile.in $(top_builddir)/config.status
+	@case '$?' in \
+	  *config.status*) \
+	    cd $(top_builddir) && $(MAKE) $(AM_MAKEFLAGS) am--refresh;; \
+	  *) \
+	    echo ' cd $(top_builddir) && $(SHELL) ./config.status $(subdir)/$@ $(am__depfiles_maybe)'; \
+	    cd $(top_builddir) && $(SHELL) ./config.status $(subdir)/$@ $(am__depfiles_maybe);; \
+	esac;
+
+$(top_builddir)/config.status: $(top_srcdir)/configure $(CONFIG_STATUS_DEPENDENCIES)
+	cd $(top_builddir) && $(MAKE) $(AM_MAKEFLAGS) am--refresh
+
+$(top_srcdir)/configure:  $(am__configure_deps)
+	cd $(top_builddir) && $(MAKE) $(AM_MAKEFLAGS) am--refresh
+$(ACLOCAL_M4):  $(am__aclocal_m4_deps)
+	cd $(top_builddir) && $(MAKE) $(AM_MAKEFLAGS) am--refresh
+
+clean-noinstLTLIBRARIES:
+	-test -z "$(noinst_LTLIBRARIES)" || rm -f $(noinst_LTLIBRARIES)
+	@list='$(noinst_LTLIBRARIES)'; for p in $$list; do \
+	  dir="`echo $$p | sed -e 's|/[^/]*$$||'`"; \
+	  test "$$dir" != "$$p" || dir=.; \
+	  echo "rm -f \"$${dir}/so_locations\""; \
+	  rm -f "$${dir}/so_locations"; \
+	done
+libnativelzo.la: $(libnativelzo_la_OBJECTS) $(libnativelzo_la_DEPENDENCIES) 
+	$(LINK)  $(libnativelzo_la_LDFLAGS) $(libnativelzo_la_OBJECTS) $(libnativelzo_la_LIBADD) $(LIBS)
+
+mostlyclean-compile:
+	-rm -f *.$(OBJEXT)
+
+distclean-compile:
+	-rm -f *.tab.c
+
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/LzoCompressor.Plo@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/LzoDecompressor.Plo@am__quote@
+
+.c.o:
+@am__fastdepCC_TRUE@	if $(COMPILE) -MT $@ -MD -MP -MF "$(DEPDIR)/$*.Tpo" -c -o $@ $<; \
+@am__fastdepCC_TRUE@	then mv -f "$(DEPDIR)/$*.Tpo" "$(DEPDIR)/$*.Po"; else rm -f "$(DEPDIR)/$*.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	source='$<' object='$@' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@	$(COMPILE) -c $<
+
+.c.obj:
+@am__fastdepCC_TRUE@	if $(COMPILE) -MT $@ -MD -MP -MF "$(DEPDIR)/$*.Tpo" -c -o $@ `$(CYGPATH_W) '$<'`; \
+@am__fastdepCC_TRUE@	then mv -f "$(DEPDIR)/$*.Tpo" "$(DEPDIR)/$*.Po"; else rm -f "$(DEPDIR)/$*.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	source='$<' object='$@' libtool=no @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@	$(COMPILE) -c `$(CYGPATH_W) '$<'`
+
+.c.lo:
+@am__fastdepCC_TRUE@	if $(LTCOMPILE) -MT $@ -MD -MP -MF "$(DEPDIR)/$*.Tpo" -c -o $@ $<; \
+@am__fastdepCC_TRUE@	then mv -f "$(DEPDIR)/$*.Tpo" "$(DEPDIR)/$*.Plo"; else rm -f "$(DEPDIR)/$*.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	source='$<' object='$@' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@	DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@	$(LTCOMPILE) -c -o $@ $<
+
+mostlyclean-libtool:
+	-rm -f *.lo
+
+clean-libtool:
+	-rm -rf .libs _libs
+
+distclean-libtool:
+	-rm -f libtool
+uninstall-info-am:
+
+ID: $(HEADERS) $(SOURCES) $(LISP) $(TAGS_FILES)
+	list='$(SOURCES) $(HEADERS) $(LISP) $(TAGS_FILES)'; \
+	unique=`for i in $$list; do \
+	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
+	  done | \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
+	mkid -fID $$unique
+tags: TAGS
+
+TAGS:  $(HEADERS) $(SOURCES)  $(TAGS_DEPENDENCIES) \
+		$(TAGS_FILES) $(LISP)
+	tags=; \
+	here=`pwd`; \
+	list='$(SOURCES) $(HEADERS)  $(LISP) $(TAGS_FILES)'; \
+	unique=`for i in $$list; do \
+	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
+	  done | \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
+	if test -z "$(ETAGS_ARGS)$$tags$$unique"; then :; else \
+	  test -n "$$unique" || unique=$$empty_fix; \
+	  $(ETAGS) $(ETAGSFLAGS) $(AM_ETAGSFLAGS) $(ETAGS_ARGS) \
+	    $$tags $$unique; \
+	fi
+ctags: CTAGS
+CTAGS:  $(HEADERS) $(SOURCES)  $(TAGS_DEPENDENCIES) \
+		$(TAGS_FILES) $(LISP)
+	tags=; \
+	here=`pwd`; \
+	list='$(SOURCES) $(HEADERS)  $(LISP) $(TAGS_FILES)'; \
+	unique=`for i in $$list; do \
+	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
+	  done | \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
+	test -z "$(CTAGS_ARGS)$$tags$$unique" \
+	  || $(CTAGS) $(CTAGSFLAGS) $(AM_CTAGSFLAGS) $(CTAGS_ARGS) \
+	     $$tags $$unique
+
+GTAGS:
+	here=`$(am__cd) $(top_builddir) && pwd` \
+	  && cd $(top_srcdir) \
+	  && gtags -i $(GTAGS_ARGS) $$here
+
+distclean-tags:
+	-rm -f TAGS ID GTAGS GRTAGS GSYMS GPATH tags
+
+distdir: $(DISTFILES)
+	@srcdirstrip=`echo "$(srcdir)" | sed 's|.|.|g'`; \
+	topsrcdirstrip=`echo "$(top_srcdir)" | sed 's|.|.|g'`; \
+	list='$(DISTFILES)'; for file in $$list; do \
+	  case $$file in \
+	    $(srcdir)/*) file=`echo "$$file" | sed "s|^$$srcdirstrip/||"`;; \
+	    $(top_srcdir)/*) file=`echo "$$file" | sed "s|^$$topsrcdirstrip/|$(top_builddir)/|"`;; \
+	  esac; \
+	  if test -f $$file || test -d $$file; then d=.; else d=$(srcdir); fi; \
+	  dir=`echo "$$file" | sed -e 's,/[^/]*$$,,'`; \
+	  if test "$$dir" != "$$file" && test "$$dir" != "."; then \
+	    dir="/$$dir"; \
+	    $(mkdir_p) "$(distdir)$$dir"; \
+	  else \
+	    dir=''; \
+	  fi; \
+	  if test -d $$d/$$file; then \
+	    if test -d $(srcdir)/$$file && test $$d != $(srcdir); then \
+	      cp -pR $(srcdir)/$$file $(distdir)$$dir || exit 1; \
+	    fi; \
+	    cp -pR $$d/$$file $(distdir)$$dir || exit 1; \
+	  else \
+	    test -f $(distdir)/$$file \
+	    || cp -p $$d/$$file $(distdir)/$$file \
+	    || exit 1; \
+	  fi; \
+	done
+check-am: all-am
+check: check-am
+all-am: Makefile $(LTLIBRARIES)
+installdirs:
+install: install-am
+install-exec: install-exec-am
+install-data: install-data-am
+uninstall: uninstall-am
+
+install-am: all-am
+	@$(MAKE) $(AM_MAKEFLAGS) install-exec-am install-data-am
+
+installcheck: installcheck-am
+install-strip:
+	$(MAKE) $(AM_MAKEFLAGS) INSTALL_PROGRAM="$(INSTALL_STRIP_PROGRAM)" \
+	  install_sh_PROGRAM="$(INSTALL_STRIP_PROGRAM)" INSTALL_STRIP_FLAG=-s \
+	  `test -z '$(STRIP)' || \
+	    echo "INSTALL_PROGRAM_ENV=STRIPPROG='$(STRIP)'"` install
+mostlyclean-generic:
+
+clean-generic:
+
+distclean-generic:
+	-test -z "$(CONFIG_CLEAN_FILES)" || rm -f $(CONFIG_CLEAN_FILES)
+
+maintainer-clean-generic:
+	@echo "This command is intended for maintainers to use"
+	@echo "it deletes files that may require special tools to rebuild."
+clean: clean-am
+
+clean-am: clean-generic clean-libtool clean-noinstLTLIBRARIES \
+	mostlyclean-am
+
+distclean: distclean-am
+	-rm -rf ./$(DEPDIR)
+	-rm -f Makefile
+distclean-am: clean-am distclean-compile distclean-generic \
+	distclean-libtool distclean-tags
+
+dvi: dvi-am
+
+dvi-am:
+
+html: html-am
+
+info: info-am
+
+info-am:
+
+install-data-am:
+
+install-exec-am:
+
+install-info: install-info-am
+
+install-man:
+
+installcheck-am:
+
+maintainer-clean: maintainer-clean-am
+	-rm -rf ./$(DEPDIR)
+	-rm -f Makefile
+maintainer-clean-am: distclean-am maintainer-clean-generic
+
+mostlyclean: mostlyclean-am
+
+mostlyclean-am: mostlyclean-compile mostlyclean-generic \
+	mostlyclean-libtool
+
+pdf: pdf-am
+
+pdf-am:
+
+ps: ps-am
+
+ps-am:
+
+uninstall-am: uninstall-info-am
+
+.PHONY: CTAGS GTAGS all all-am check check-am clean clean-generic \
+	clean-libtool clean-noinstLTLIBRARIES ctags distclean \
+	distclean-compile distclean-generic distclean-libtool \
+	distclean-tags distdir dvi dvi-am html html-am info info-am \
+	install install-am install-data install-data-am install-exec \
+	install-exec-am install-info install-info-am install-man \
+	install-strip installcheck installcheck-am installdirs \
+	maintainer-clean maintainer-clean-generic mostlyclean \
+	mostlyclean-compile mostlyclean-generic mostlyclean-libtool \
+	pdf pdf-am ps ps-am tags uninstall uninstall-am \
+	uninstall-info-am
+
+
+# The 'vpath directive' to locate the actual source files 
+vpath %.c $(HADOOP_NATIVE_SRCDIR)/$(subdir)
+
+#
+#vim: sw=4: ts=4: noet
+#
+# Tell versions [3.59,3.63) of GNU make to not export all variables.
+# Otherwise a system limit (for SysV at least) may be exceeded.
+.NOEXPORT:

+ 112 - 0
src/native/src/org/apache/hadoop/io/compress/lzo/org_apache_hadoop_io_compress_lzo.h

@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#if !defined ORG_APACHE_HADOOP_IO_COMPRESS_LZO_LZO_H
+#define ORG_APACHE_HADOOP_IO_COMPRESS_LZO_LZO_H
+
+#if defined HAVE_CONFIG_H
+  #include <config.h>
+#endif
+
+#if defined HAVE_STDDEF_H
+  #include <stddef.h>
+#else
+  #error 'stddef.h not found'
+#endif
+    
+#if defined HAVE_DLFCN_H
+  #include <dlfcn.h>
+#else
+  #error "dlfcn.h not found"
+#endif  
+
+#if defined HAVE_JNI_H    
+  #include <jni.h>
+#else
+  #error 'jni.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1_H
+  #include <lzo/lzo1.h>
+#else
+  #error 'lzo/lzo1.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1A_H
+  #include <lzo/lzo1a.h>
+#else
+  #error 'lzo/lzo1a.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1B_H
+  #include <lzo/lzo1b.h>
+#else
+  #error 'lzo/lzo1b.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1C_H
+  #include <lzo/lzo1c.h>
+#else
+  #error 'lzo/lzo1c.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1F_H
+  #include <lzo/lzo1f.h>
+#else
+  #error 'lzo/lzo1f.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1X_H
+  #include <lzo/lzo1x.h>
+#else
+  #error 'lzo/lzo1x.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1Y_H
+  #include <lzo/lzo1y.h>
+#else
+  #error 'lzo/lzo1y.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO1Z_H
+  #include <lzo/lzo1z.h>
+#else
+  #error 'lzo/lzo1z.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO2A_H
+  #include <lzo/lzo2a.h>
+#else
+  #error 'lzo/lzo2a.h not found'
+#endif
+
+#if defined HAVE_LZO_LZO_ASM_H
+  #include <lzo/lzo_asm.h>
+#else
+  #error 'lzo/lzo_asm.h not found'
+#endif
+
+#include "org_apache_hadoop.h"
+
+/* A helper macro to convert the java 'function-pointer' to a void*. */
+#define FUNC_PTR(func_ptr) ((void*)((ptrdiff_t)(func_ptr)))
+
+/* A helper macro to convert the void* to the java 'function-pointer'. */
+#define JLONG(func_ptr) ((jlong)((ptrdiff_t)(func_ptr)))
+
+#endif //ORG_APACHE_HADOOP_IO_COMPRESS_LZO_LZO_H

+ 5 - 5
src/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c

@@ -75,11 +75,11 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibCompressor_initIDs(
 
 	// Locate the requisite symbols from libz.so
 	dlerror();                                 // Clear any existing error
-	LOAD_ZLIB_SYMBOL(dlsym_deflateInit2_, env, libz, "deflateInit2_");
-	LOAD_ZLIB_SYMBOL(dlsym_deflate, env, libz, "deflate");
-	LOAD_ZLIB_SYMBOL(dlsym_deflateSetDictionary, env, libz, "deflateSetDictionary");
-	LOAD_ZLIB_SYMBOL(dlsym_deflateReset, env, libz, "deflateReset");
-	LOAD_ZLIB_SYMBOL(dlsym_deflateEnd, env, libz, "deflateEnd");
+	LOAD_DYNAMIC_SYMBOL(dlsym_deflateInit2_, env, libz, "deflateInit2_");
+	LOAD_DYNAMIC_SYMBOL(dlsym_deflate, env, libz, "deflate");
+	LOAD_DYNAMIC_SYMBOL(dlsym_deflateSetDictionary, env, libz, "deflateSetDictionary");
+	LOAD_DYNAMIC_SYMBOL(dlsym_deflateReset, env, libz, "deflateReset");
+	LOAD_DYNAMIC_SYMBOL(dlsym_deflateEnd, env, libz, "deflateEnd");
 
 	// Initialize the requisite fieldIds
     ZlibCompressor_stream = (*env)->GetFieldID(env, class, "stream", "J");

+ 5 - 5
src/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c

@@ -75,11 +75,11 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibDecompressor_initIDs(
 
 	// Locate the requisite symbols from libz.so
 	dlerror();                                 // Clear any existing error
-	LOAD_ZLIB_SYMBOL(dlsym_inflateInit2_, env, libz, "inflateInit2_");
-	LOAD_ZLIB_SYMBOL(dlsym_inflate, env, libz, "inflate");
-	LOAD_ZLIB_SYMBOL(dlsym_inflateSetDictionary, env, libz, "inflateSetDictionary");
-	LOAD_ZLIB_SYMBOL(dlsym_inflateReset, env, libz, "inflateReset");
-	LOAD_ZLIB_SYMBOL(dlsym_inflateEnd, env, libz, "inflateEnd");
+	LOAD_DYNAMIC_SYMBOL(dlsym_inflateInit2_, env, libz, "inflateInit2_");
+	LOAD_DYNAMIC_SYMBOL(dlsym_inflate, env, libz, "inflate");
+	LOAD_DYNAMIC_SYMBOL(dlsym_inflateSetDictionary, env, libz, "inflateSetDictionary");
+	LOAD_DYNAMIC_SYMBOL(dlsym_inflateReset, env, libz, "inflateReset");
+	LOAD_DYNAMIC_SYMBOL(dlsym_inflateEnd, env, libz, "inflateEnd");
 
 	// Initialize the requisite fieldIds
     ZlibDecompressor_stream = (*env)->GetFieldID(env, class, "stream", "J");

+ 0 - 6
src/native/src/org/apache/hadoop/io/compress/zlib/org_apache_hadoop_io_compress_zlib.h

@@ -55,12 +55,6 @@
 
 #include "org_apache_hadoop.h"
 
-/* A helper macro to dlsym the requisite zlib symbol. */
-#define LOAD_ZLIB_SYMBOL(func_ptr, env, handle, symbol) \
-  if ((func_ptr = do_dlsym(env, handle, symbol)) == NULL) { \
-  	return; \
-  }
-
 /* A helper macro to convert the java 'stream-handle' to a z_stream pointer. */
 #define ZSTREAM(stream) ((z_stream*)((ptrdiff_t)(stream)))
 

+ 7 - 0
src/native/src/org_apache_hadoop.h

@@ -73,6 +73,13 @@ static void *do_dlsym(JNIEnv *env, void *handle, const char *symbol) {
   return func_ptr;
 }
 
+/* A helper macro to dlsym the requisite dynamic symbol and bail-out on error. */
+#define LOAD_DYNAMIC_SYMBOL(func_ptr, env, handle, symbol) \
+  if ((func_ptr = do_dlsym(env, handle, symbol)) == NULL) { \
+    return; \
+  }
+
+
 #endif
 
 //vim: sw=2: ts=2: et

+ 47 - 14
src/test/org/apache/hadoop/fs/s3/S3FileSystemBaseTest.java

@@ -166,9 +166,7 @@ public abstract class S3FileSystemBaseTest extends TestCase {
     
     s3FileSystem.mkdirs(path.getParent());
 
-    FSOutputStream out = s3FileSystem.createRaw(path, false, (short) 1, BLOCK_SIZE);
-    out.write(data, 0, BLOCK_SIZE);
-    out.close();
+    createEmptyFile(path);
     
     assertTrue("Exists", s3FileSystem.exists(path));
     assertEquals("Length", BLOCK_SIZE, s3FileSystem.getLength(path));
@@ -180,7 +178,7 @@ public abstract class S3FileSystemBaseTest extends TestCase {
       // Expected
     }
     
-    out = s3FileSystem.createRaw(path, true, (short) 1, BLOCK_SIZE);
+    FSOutputStream out = s3FileSystem.createRaw(path, true, (short) 1, BLOCK_SIZE);
     out.write(data, 0, BLOCK_SIZE / 2);
     out.close();
     
@@ -189,14 +187,46 @@ public abstract class S3FileSystemBaseTest extends TestCase {
     
   }
 
-  public void testWriteInNonExistentDirectory() {
+  public void testWriteInNonExistentDirectory() throws IOException {
     Path path = new Path("/test/hadoop/file");    
-    try {
-      s3FileSystem.createRaw(path, false, (short) 1, 128);
-      fail("Should throw IOException.");
-    } catch (IOException e) {
-      // Expected
-    }
+    createEmptyFile(path);
+    
+    assertTrue("Exists", s3FileSystem.exists(path));
+    assertEquals("Length", BLOCK_SIZE, s3FileSystem.getLength(path));
+    assertTrue("Parent exists", s3FileSystem.exists(path.getParent()));
+  }
+
+  public void testDeleteNonExistentFile() throws IOException {
+    Path path = new Path("/test/hadoop/file");    
+    assertFalse("Doesn't exist", s3FileSystem.exists(path));
+    assertFalse("No deletion", s3FileSystem.delete(path));
+  }
+
+  public void testDeleteDirectory() throws IOException {
+    Path subdir = new Path("/test/hadoop");
+    Path dir = subdir.getParent();
+    Path root = dir.getParent();
+    s3FileSystem.mkdirs(subdir);
+    Path file1 = new Path(dir, "file1");
+    Path file2 = new Path(subdir, "file2");
+    
+    createEmptyFile(file1);
+    createEmptyFile(file2);
+    
+    assertTrue("root exists", s3FileSystem.exists(root));
+    assertTrue("dir exists", s3FileSystem.exists(dir));
+    assertTrue("file1 exists", s3FileSystem.exists(file1));
+    assertTrue("subdir exists", s3FileSystem.exists(subdir));
+    assertTrue("file2 exists", s3FileSystem.exists(file2));
+    
+    assertTrue("Delete", s3FileSystem.delete(dir));
+
+    assertTrue("root exists", s3FileSystem.exists(root));
+    assertFalse("dir exists", s3FileSystem.exists(dir));
+    assertFalse("file1 exists", s3FileSystem.exists(file1));
+    assertFalse("subdir exists", s3FileSystem.exists(subdir));
+    assertFalse("file2 exists", s3FileSystem.exists(file2));
+    
   }
 
   public void testRename() throws Exception {
@@ -206,9 +236,7 @@ public abstract class S3FileSystemBaseTest extends TestCase {
     
     s3FileSystem.mkdirs(path.getParent());
 
-    FSOutputStream out = s3FileSystem.createRaw(path, false, (short) 1, BLOCK_SIZE);
-    out.write(data, 0, len);
-    out.close();
+    createEmptyFile(path);
 
     assertTrue("Exists", s3FileSystem.exists(path));
 
@@ -228,5 +256,10 @@ public abstract class S3FileSystemBaseTest extends TestCase {
     }
   }
 
+  private void createEmptyFile(Path path) throws IOException {
+    FSOutputStream out = s3FileSystem.createRaw(path, false, (short) 1, BLOCK_SIZE);
+    out.write(data, 0, BLOCK_SIZE);
+    out.close();
+  }
 
 }

+ 20 - 3
src/test/org/apache/hadoop/io/TestSequenceFile.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.compress.LzoCodec;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.conf.*;
 
@@ -41,8 +42,26 @@ public class TestSequenceFile extends TestCase {
   public TestSequenceFile(String name) { super(name); }
 
   /** Unit tests for SequenceFile. */
-  public void testSequenceFile() throws Exception {
+  public void testZlibSequenceFile() throws Exception {
+    LOG.info("Testing SequenceFile with DefaultCodec");
     compressedSeqFileTest(new DefaultCodec());
+    LOG.info("Successfully tested SequenceFile with DefaultCodec");
+  }
+  
+  public void testLzoSequenceFile() throws Exception {
+    if (LzoCodec.isNativeLzoLoaded()) {
+      LOG.info("Testing SequenceFile with LzoCodec");
+      CompressionCodec lzoCodec = null;
+      try {
+        lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(
+                conf.getClassByName(LzoCodec.class.getName()), conf);
+      } catch (ClassNotFoundException cnfe) {
+        throw new IOException("Cannot find LzoCodec!");
+      }
+
+      compressedSeqFileTest(lzoCodec);
+      LOG.info("Successfully tested SequenceFile with LzoCodec");
+    }
   }
   
   public void compressedSeqFileTest(CompressionCodec codec) throws Exception {
@@ -60,8 +79,6 @@ public class TestSequenceFile extends TestCase {
 
     FileSystem fs = new LocalFileSystem(conf);
     try {
-        //LOG.setLevel(Level.FINE);
-
         // SequenceFile.Writer
         writeTest(fs, count, seed, file, CompressionType.NONE, null);
         readTest(fs, count, seed, file);

+ 6 - 0
src/test/org/apache/hadoop/io/compress/TestCodec.java

@@ -51,6 +51,12 @@ public class TestCodec extends TestCase {
     codecTest(seed, count, "org.apache.hadoop.io.compress.GzipCodec");
   }
   
+  public void testLzoCodec() throws IOException {
+    if (LzoCodec.isNativeLzoLoaded()) {
+      codecTest(seed, count, "org.apache.hadoop.io.compress.LzoCodec");
+    }
+  }
+  
   private static void codecTest(int seed, int count, String codecClass) 
   throws IOException {
     

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMapRed.java

@@ -455,7 +455,7 @@ public class TestMapRed extends TestCase {
         checkJob.setOutputPath(intermediateOuts);
         checkJob.setOutputKeyClass(IntWritable.class);
         checkJob.setOutputValueClass(IntWritable.class);
-        checkJob.setOutputFormat(SequenceFileOutputFormat.class);
+        checkJob.setOutputFormat(MapFileOutputFormat.class);
         checkJob.setReducerClass(RandomCheckReducer.class);
         checkJob.setNumReduceTasks(intermediateReduces);
 

Деякі файли не було показано, через те що забагато файлів було змінено