浏览代码

HADOOP-4874. Remove LZO codec because of licensing issues. (omalley)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@727294 13f79535-47bb-0310-9956-ffa450edef68
Owen O'Malley 16 年之前
父节点
当前提交
efa3660924
共有 26 个文件被更改,包括 381 次插入4032 次删除
  1. 2 0
      CHANGES.txt
  2. 0 198
      src/core/org/apache/hadoop/io/compress/LzoCodec.java
  3. 0 502
      src/core/org/apache/hadoop/io/compress/LzopCodec.java
  4. 0 359
      src/core/org/apache/hadoop/io/compress/lzo/LzoCompressor.java
  5. 0 321
      src/core/org/apache/hadoop/io/compress/lzo/LzoDecompressor.java
  6. 6 6
      src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
  7. 5 9
      src/docs/src/documentation/content/xdocs/native_libraries.xml
  8. 0 1
      src/docs/src/documentation/content/xdocs/site.xml
  9. 0 385
      src/mapred/org/apache/hadoop/mapred/LzoTextInputFormat.java
  10. 1 1
      src/native/Makefile.am
  11. 7 20
      src/native/Makefile.in
  12. 151 356
      src/native/aclocal.m4
  13. 0 33
      src/native/config.h.in
  14. 193 401
      src/native/configure
  15. 0 4
      src/native/configure.ac
  16. 6 5
      src/native/lib/Makefile.in
  17. 0 301
      src/native/src/org/apache/hadoop/io/compress/lzo/LzoCompressor.c
  18. 0 238
      src/native/src/org/apache/hadoop/io/compress/lzo/LzoDecompressor.c
  19. 0 50
      src/native/src/org/apache/hadoop/io/compress/lzo/Makefile.am
  20. 0 469
      src/native/src/org/apache/hadoop/io/compress/lzo/Makefile.in
  21. 0 112
      src/native/src/org/apache/hadoop/io/compress/lzo/org_apache_hadoop_io_compress_lzo.h
  22. 6 5
      src/native/src/org/apache/hadoop/io/compress/zlib/Makefile.in
  23. 4 5
      src/test/org/apache/hadoop/io/FileBench.java
  24. 0 17
      src/test/org/apache/hadoop/io/TestSequenceFile.java
  25. 0 12
      src/test/org/apache/hadoop/io/compress/TestCodec.java
  26. 0 222
      src/test/org/apache/hadoop/mapred/TestLzoTextInputFormat.java

+ 2 - 0
CHANGES.txt

@@ -74,6 +74,8 @@ Release 0.20.0 - Unreleased
     pipes, utils, and libhdfs are now all in c++/<os_osarch_jvmdatamodel>/lib. 
     (Giridharan Kesavan via nigel)
 
+    HADOOP-4874. Remove LZO codec because of licensing issues. (omalley)
+
   NEW FEATURES
 
     HADOOP-4575. Add a proxy service for relaying HsftpFileSystem requests.

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

@@ -1,198 +0,0 @@
-/*
- * 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/
- * 
- */
-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.
-   * 
-   * @param conf configuration
-   * @return <code>true</code> if native-lzo library is loaded & initialized;
-   *         else <code>false</code>
-   */
-  public static boolean isNativeLzoLoaded(Configuration conf) {
-    return nativeLzoLoaded && conf.getBoolean("hadoop.native.lib", true);
-  }
-
-  public CompressionOutputStream createOutputStream(OutputStream out) 
-    throws IOException {
-    return createOutputStream(out, createCompressor());
-  }
-  
-  public CompressionOutputStream createOutputStream(OutputStream out, 
-      Compressor compressor) throws IOException {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(conf)) {
-      throw new RuntimeException("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 = strategy.name().contains("LZO1")
-      ? (bufferSize >> 4) + 64 + 3
-      : (bufferSize >> 3) + 128 + 3;
-
-    return new BlockCompressorStream(out, compressor, bufferSize,
-                                     compressionOverhead);
-  }
-
-  public Class<? extends Compressor> getCompressorType() {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(conf)) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    return LzoCompressor.class;
-  }
-
-  public Compressor createCompressor() {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(conf)) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    
-    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);
-
-    return new LzoCompressor(strategy, bufferSize);
-  }
-
-  public CompressionInputStream createInputStream(InputStream in)
-      throws IOException {
-    return createInputStream(in, createDecompressor());
-  }
-
-  public CompressionInputStream createInputStream(InputStream in, 
-                                                  Decompressor decompressor) 
-  throws IOException {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(conf)) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    return new BlockDecompressorStream(in, decompressor, 
-        conf.getInt("io.compression.codec.lzo.buffersize", 64*1024));
-  }
-
-  public Class<? extends Decompressor> getDecompressorType() {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(conf)) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    return LzoDecompressor.class;
-  }
-
-  public Decompressor createDecompressor() {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(conf)) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    
-    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 LzoDecompressor(strategy, bufferSize); 
-  }
-
-  /**
-   * Get the default filename extension for this kind of compression.
-   * @return the extension including the '.'
-   */
-  public String getDefaultExtension() {
-    return ".lzo_deflate";
-  }
-}

+ 0 - 502
src/core/org/apache/hadoop/io/compress/LzopCodec.java

@@ -1,502 +0,0 @@
-/*
- * 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;
-import java.io.OutputStream;
-import java.io.PushbackInputStream;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.EnumMap;
-import java.util.Map;
-import java.util.zip.Adler32;
-import java.util.zip.Checksum;
-import java.util.zip.CRC32;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.compress.lzo.*;
-import org.apache.hadoop.util.NativeCodeLoader;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * A {@link org.apache.hadoop.io.compress.CompressionCodec} for a streaming
- * <b>lzo</b> compression/decompression pair compatible with lzop.
- * http://www.lzop.org/
- */
-public class LzopCodec extends LzoCodec {
-
-  private static final Log LOG = LogFactory.getLog(LzopCodec.class.getName());
-  /** 9 bytes at the top of every lzo file */
-  private static final byte[] LZO_MAGIC = new byte[] {
-    -119, 'L', 'Z', 'O', 0, '\r', '\n', '\032', '\n' };
-  /** Version of lzop this emulates */
-  private static final int LZOP_VERSION = 0x1010;
-  /** Latest verion of lzop this should be compatible with */
-  private static final int LZOP_COMPAT_VERSION = 0x0940;
-
-  public CompressionOutputStream createOutputStream(OutputStream out,
-      Compressor compressor) throws IOException {
-    if (!isNativeLzoLoaded(getConf())) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    LzoCompressor.CompressionStrategy strategy =
-      LzoCompressor.CompressionStrategy.valueOf(
-          getConf().get("io.compression.codec.lzo.compressor",
-            LzoCompressor.CompressionStrategy.LZO1X_1.name()));
-    int bufferSize =
-      getConf().getInt("io.compression.codec.lzo.buffersize", 64*1024);
-    return new LzopOutputStream(out, compressor, bufferSize, strategy);
-  }
-
-  public CompressionInputStream createInputStream(InputStream in,
-      Decompressor decompressor) throws IOException {
-    // Ensure native-lzo library is loaded & initialized
-    if (!isNativeLzoLoaded(getConf())) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    return new LzopInputStream(in, decompressor,
-        getConf().getInt("io.compression.codec.lzo.buffersize", 256 * 1024));
-  }
-
-  public Decompressor createDecompressor() {
-    if (!isNativeLzoLoaded(getConf())) {
-      throw new RuntimeException("native-lzo library not available");
-    }
-    return new LzopDecompressor(getConf().getInt(
-          "io.compression.codec.lzo.buffersize", 256 * 1024));
-  }
-
-  public String getDefaultExtension() {
-    return ".lzo";
-  }
-
-  /**
-   * Checksums on decompressed block data with header bitmask, Checksum class.
-   */
-  private enum DChecksum {
-    F_ADLER32D(0x01, Adler32.class), F_CRC32D(0x100, CRC32.class);
-    private int mask;
-    private Class<? extends Checksum> clazz;
-    DChecksum(int mask, Class<? extends Checksum> clazz) {
-      this.mask = mask;
-      this.clazz = clazz;
-    }
-    public int getHeaderMask() {
-      return mask;
-    }
-    public Class<? extends Checksum> getChecksumClass() {
-      return clazz;
-    }
-  }
-
-  /**
-   * Checksums on compressed block data with header bitmask, Checksum class.
-   */
-  private enum CChecksum {
-    F_ADLER32C(0x02, Adler32.class), F_CRC32C(0x200, CRC32.class);
-    private int mask;
-    private Class<? extends Checksum> clazz;
-    CChecksum(int mask, Class<? extends Checksum> clazz) {
-      this.mask = mask;
-      this.clazz = clazz;
-    }
-    public int getHeaderMask() {
-      return mask;
-    }
-    public Class<? extends Checksum> getChecksumClass() {
-      return clazz;
-    }
-  };
-
-  protected static class LzopOutputStream extends BlockCompressorStream {
-
-    /**
-     * Write an lzop-compatible header to the OutputStream provided.
-     */
-    protected static void writeLzopHeader(OutputStream out,
-        LzoCompressor.CompressionStrategy strategy) throws IOException {
-      DataOutputBuffer dob = new DataOutputBuffer();
-      try {
-        dob.writeShort(LZOP_VERSION);
-        dob.writeShort(LzoCompressor.LZO_LIBRARY_VERSION);
-        dob.writeShort(LZOP_COMPAT_VERSION);
-        switch (strategy) {
-          case LZO1X_1:
-            dob.writeByte(1);
-            dob.writeByte(5);
-            break;
-          case LZO1X_15:
-            dob.writeByte(2);
-            dob.writeByte(1);
-            break;
-          case LZO1X_999:
-            dob.writeByte(3);
-            dob.writeByte(9);
-            break;
-          default:
-            throw new IOException("Incompatible lzop strategy: " + strategy);
-        }
-        dob.writeInt(0);                                    // all flags 0
-        dob.writeInt(0x81A4);                               // mode
-        dob.writeInt((int)(System.currentTimeMillis() / 1000)); // mtime
-        dob.writeInt(0);                                    // gmtdiff ignored
-        dob.writeByte(0);                                   // no filename
-        Adler32 headerChecksum = new Adler32();
-        headerChecksum.update(dob.getData(), 0, dob.getLength());
-        int hc = (int)headerChecksum.getValue();
-        dob.writeInt(hc);
-        out.write(LZO_MAGIC);
-        out.write(dob.getData(), 0, dob.getLength());
-      } finally {
-        dob.close();
-      }
-    }
-
-    public LzopOutputStream(OutputStream out, Compressor compressor,
-        int bufferSize, LzoCompressor.CompressionStrategy strategy)
-        throws IOException {
-      super(out, compressor, bufferSize, strategy.name().contains("LZO1")
-          ? (bufferSize >> 4) + 64 + 3
-          : (bufferSize >> 3) + 128 + 3);
-      writeLzopHeader(out, strategy);
-    }
-
-    /**
-     * Close the underlying stream and write a null word to the output stream.
-     */
-    public void close() throws IOException {
-      if (!closed) {
-        finish();
-        out.write(new byte[]{ 0, 0, 0, 0 });
-        out.close();
-        closed = true;
-      }
-    }
-
-  }
-
-  protected static class LzopInputStream extends BlockDecompressorStream {
-
-    private EnumSet<DChecksum> dflags = EnumSet.allOf(DChecksum.class);
-    private EnumSet<CChecksum> cflags = EnumSet.allOf(CChecksum.class);
-
-    private final byte[] buf = new byte[9];
-    private EnumMap<DChecksum,Integer> dcheck
-      = new EnumMap<DChecksum,Integer>(DChecksum.class);
-    private EnumMap<CChecksum,Integer> ccheck
-      = new EnumMap<CChecksum,Integer>(CChecksum.class);
-
-    public LzopInputStream(InputStream in, Decompressor decompressor,
-        int bufferSize) throws IOException {
-      super(in, decompressor, bufferSize);
-      readHeader(in);
-    }
-
-    /**
-     * Read len bytes into buf, st LSB of int returned is the last byte of the
-     * first word read.
-     */
-    private static int readInt(InputStream in, byte[] buf, int len) 
-        throws IOException {
-      if (0 > in.read(buf, 0, len)) {
-        throw new EOFException();
-      }
-      int ret = (0xFF & buf[0]) << 24;
-      ret    |= (0xFF & buf[1]) << 16;
-      ret    |= (0xFF & buf[2]) << 8;
-      ret    |= (0xFF & buf[3]);
-      return (len > 3) ? ret : (ret >>> (8 * (4 - len)));
-    }
-
-    /**
-     * Read bytes, update checksums, return first four bytes as an int, first
-     * byte read in the MSB.
-     */
-    private static int readHeaderItem(InputStream in, byte[] buf, int len,
-        Adler32 adler, CRC32 crc32) throws IOException {
-      int ret = readInt(in, buf, len);
-      adler.update(buf, 0, len);
-      crc32.update(buf, 0, len);
-      Arrays.fill(buf, (byte)0);
-      return ret;
-    }
-
-    /**
-     * Read and verify an lzo header, setting relevant block checksum options
-     * and ignoring most everything else.
-     */
-    protected void readHeader(InputStream in) throws IOException {
-      if (0 > in.read(buf, 0, 9)) {
-        throw new EOFException();
-      }
-      if (!Arrays.equals(buf, LZO_MAGIC)) {
-        throw new IOException("Invalid LZO header");
-      }
-      Arrays.fill(buf, (byte)0);
-      Adler32 adler = new Adler32();
-      CRC32 crc32 = new CRC32();
-      int hitem = readHeaderItem(in, buf, 2, adler, crc32); // lzop version
-      if (hitem > LZOP_VERSION) {
-        LOG.debug("Compressed with later version of lzop: " +
-            Integer.toHexString(hitem) + " (expected 0x" +
-            Integer.toHexString(LZOP_VERSION) + ")");
-      }
-      hitem = readHeaderItem(in, buf, 2, adler, crc32); // lzo library version
-      if (hitem > LzoDecompressor.LZO_LIBRARY_VERSION) {
-        throw new IOException("Compressed with incompatible lzo version: 0x" +
-            Integer.toHexString(hitem) + " (expected 0x" +
-            Integer.toHexString(LzoDecompressor.LZO_LIBRARY_VERSION) + ")");
-      }
-      hitem = readHeaderItem(in, buf, 2, adler, crc32); // lzop extract version
-      if (hitem > LZOP_VERSION) {
-        throw new IOException("Compressed with incompatible lzop version: 0x" +
-            Integer.toHexString(hitem) + " (expected 0x" +
-            Integer.toHexString(LZOP_VERSION) + ")");
-      }
-      hitem = readHeaderItem(in, buf, 1, adler, crc32); // method
-      if (hitem < 1 || hitem > 3) {
-          throw new IOException("Invalid strategy: " +
-              Integer.toHexString(hitem));
-      }
-      readHeaderItem(in, buf, 1, adler, crc32); // ignore level
-
-      // flags
-      hitem = readHeaderItem(in, buf, 4, adler, crc32);
-      try {
-        for (DChecksum f : dflags) {
-          if (0 == (f.getHeaderMask() & hitem)) {
-            dflags.remove(f);
-          } else {
-            dcheck.put(f, (int)f.getChecksumClass().newInstance().getValue());
-          }
-        }
-        for (CChecksum f : cflags) {
-          if (0 == (f.getHeaderMask() & hitem)) {
-            cflags.remove(f);
-          } else {
-            ccheck.put(f, (int)f.getChecksumClass().newInstance().getValue());
-          }
-        }
-      } catch (InstantiationException e) {
-        throw new RuntimeException("Internal error", e);
-      } catch (IllegalAccessException e) {
-        throw new RuntimeException("Internal error", e);
-      }
-      ((LzopDecompressor)decompressor).initHeaderFlags(dflags, cflags);
-      boolean useCRC32 = 0 != (hitem & 0x00001000);   // F_H_CRC32
-      boolean extraField = 0 != (hitem & 0x00000040); // F_H_EXTRA_FIELD
-      if (0 != (hitem & 0x400)) {                     // F_MULTIPART
-        throw new IOException("Multipart lzop not supported");
-      }
-      if (0 != (hitem & 0x800)) {                     // F_H_FILTER
-        throw new IOException("lzop filter not supported");
-      }
-      if (0 != (hitem & 0x000FC000)) {                // F_RESERVED
-        throw new IOException("Unknown flags in header");
-      }
-      // known !F_H_FILTER, so no optional block
-
-      readHeaderItem(in, buf, 4, adler, crc32); // ignore mode
-      readHeaderItem(in, buf, 4, adler, crc32); // ignore mtime
-      readHeaderItem(in, buf, 4, adler, crc32); // ignore gmtdiff
-      hitem = readHeaderItem(in, buf, 1, adler, crc32); // fn len
-      if (hitem > 0) {
-        // skip filename
-        readHeaderItem(in, new byte[hitem], hitem, adler, crc32);
-      }
-      int checksum = (int)(useCRC32 ? crc32.getValue() : adler.getValue());
-      hitem = readHeaderItem(in, buf, 4, adler, crc32); // read checksum
-      if (hitem != checksum) {
-        throw new IOException("Invalid header checksum: " +
-            Long.toHexString(checksum) + " (expected 0x" +
-            Integer.toHexString(hitem) + ")");
-      }
-      if (extraField) { // lzop 1.08 ultimately ignores this
-        LOG.debug("Extra header field not processed");
-        adler.reset();
-        crc32.reset();
-        hitem = readHeaderItem(in, buf, 4, adler, crc32);
-        readHeaderItem(in, new byte[hitem], hitem, adler, crc32);
-        checksum = (int)(useCRC32 ? crc32.getValue() : adler.getValue());
-        if (checksum != readHeaderItem(in, buf, 4, adler, crc32)) {
-          throw new IOException("Invalid checksum for extra header field");
-        }
-      }
-    }
-
-    /**
-     * Take checksums recorded from block header and verify them against
-     * those recorded by the decomrpessor.
-     */
-    private void verifyChecksums() throws IOException {
-      LzopDecompressor ldecompressor = ((LzopDecompressor)decompressor);
-      for (Map.Entry<DChecksum,Integer> chk : dcheck.entrySet()) {
-        if (!ldecompressor.verifyDChecksum(chk.getKey(), chk.getValue())) {
-          throw new IOException("Corrupted uncompressed block");
-        }
-      }
-      for (Map.Entry<CChecksum,Integer> chk : ccheck.entrySet()) {
-        if (!ldecompressor.verifyCChecksum(chk.getKey(), chk.getValue())) {
-          throw new IOException("Corrupted compressed block");
-        }
-      }
-    }
-
-    /**
-     * Read checksums and feed compressed block data into decompressor.
-     */
-    void getCompressedData() throws IOException {
-      checkStream();
-
-      LzopDecompressor ldecompressor = (LzopDecompressor)decompressor;
-
-      // Get the size of the compressed chunk
-      int len = readInt(in, buf, 4);
-
-      verifyChecksums();
-
-      for (DChecksum chk : dcheck.keySet()) {
-        dcheck.put(chk, readInt(in, buf, 4));
-      }
-      for (CChecksum chk : ccheck.keySet()) {
-        // NOTE: if the compressed size is not less than the uncompressed
-        //       size, this value is not present and decompression will fail.
-        //       Fortunately, checksums on compressed data are rare, as is
-        //       this case.
-        ccheck.put(chk, readInt(in, buf, 4));
-      }
-
-      ldecompressor.resetChecksum();
-
-      // 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 close() throws IOException {
-      byte[] b = new byte[4096];
-      while (!decompressor.finished()) {
-        decompressor.decompress(b, 0, b.length);
-      }
-      super.close();
-      verifyChecksums();
-    }
-  }
-
-  public static class LzopDecompressor extends LzoDecompressor {
-
-    private EnumMap<DChecksum,Checksum> chkDMap =
-      new EnumMap<DChecksum,Checksum>(DChecksum.class);
-    private EnumMap<CChecksum,Checksum> chkCMap =
-      new EnumMap<CChecksum,Checksum>(CChecksum.class);
-
-    /**
-     * Create an LzoDecompressor with LZO1X strategy (the only lzo algorithm
-     * supported by lzop).
-     */
-    public LzopDecompressor(int bufferSize) {
-      super(LzoDecompressor.CompressionStrategy.LZO1X_SAFE, bufferSize);
-    }
-
-    /**
-     * Get the number of checksum implementations
-     * the current lzo file uses.
-     * @return Number of checksum implementations in use.
-     */
-    public int getChecksumsCount() {
-      return this.chkCMap.size() + this.chkDMap.size();
-    }
-    
-    /**
-     * Given a set of decompressed and compressed checksums, 
-     */
-    public void initHeaderFlags(EnumSet<DChecksum> dflags,
-        EnumSet<CChecksum> cflags) {
-      try {
-        for (DChecksum flag : dflags) {
-          chkDMap.put(flag, flag.getChecksumClass().newInstance());
-        }
-        for (CChecksum flag : cflags) {
-          chkCMap.put(flag, flag.getChecksumClass().newInstance());
-        }
-      } catch (InstantiationException e) {
-        throw new RuntimeException("Internal error", e);
-      } catch (IllegalAccessException e) {
-        throw new RuntimeException("Internal error", e);
-      }
-    }
-
-    /**
-     * Reset all checksums registered for this decompressor instance.
-     */
-    public synchronized void resetChecksum() {
-      for (Checksum chk : chkDMap.values()) chk.reset();
-      for (Checksum chk : chkCMap.values()) chk.reset();
-    }
-
-    /**
-     * Given a checksum type, verify its value against that observed in
-     * decompressed data.
-     */
-    public synchronized boolean verifyDChecksum(DChecksum typ, int checksum) {
-      return (checksum == (int)chkDMap.get(typ).getValue());
-    }
-
-    /**
-     * Given a checksum type, verity its value against that observed in
-     * compressed data.
-     */
-    public synchronized boolean verifyCChecksum(CChecksum typ, int checksum) {
-      return (checksum == (int)chkCMap.get(typ).getValue());
-    }
-
-    public synchronized void setInput(byte[] b, int off, int len) {
-      for (Checksum chk : chkCMap.values()) chk.update(b, off, len);
-      super.setInput(b, off, len);
-    }
-
-    public synchronized int decompress(byte[] b, int off, int len)
-        throws IOException {
-      int ret = super.decompress(b, off, len);
-      if (ret > 0) {
-        for (Checksum chk : chkDMap.values()) chk.update(b, off, ret);
-      }
-      return ret;
-    }
-  }
-
-}

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

@@ -1,359 +0,0 @@
-/*
- * 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/
- * 
- */
-public class LzoCompressor implements Compressor {
-  private static final Log LOG = 
-    LogFactory.getLog(LzoCompressor.class.getName());
-
-  // HACK - Use this as a global lock in the JNI layer
-  private static Class clazz = LzoDecompressor.class;
-  
-  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 long bytesread = 0L;
-  private long byteswritten = 0L;
-
-  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;
-  public static final int LZO_LIBRARY_VERSION;
-  
-  static {
-    if (NativeCodeLoader.isNativeCodeLoaded()) {
-      // Initialize the native library
-      try {
-        initIDs();
-        nativeLzoLoaded = true;
-      } catch (Throwable t) {
-        // Ignore failure to load/initialize native-lzo
-        nativeLzoLoaded = false;
-      }
-      LZO_LIBRARY_VERSION = (nativeLzoLoaded) ? 0xFFFF & getLzoLibraryVersion()
-                                              : -1;
-    } else {
-      LOG.error("Cannot load " + LzoCompressor.class.getName() + 
-                " without native-hadoop library!");
-      nativeLzoLoaded = false;
-      LZO_LIBRARY_VERSION = -1;
-     }
-   }
-  
-  /**
-   * 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();
-    }
-    finished = false;
-
-    if (len > uncompressedDirectBuf.remaining()) {
-      // save data; now !needsInput
-      this.userBuf = b;
-      this.userBufOff = off;
-      this.userBufLen = len;
-    } else {
-      ((ByteBuffer)uncompressedDirectBuf).put(b, off, len);
-      uncompressedDirectBufLen = uncompressedDirectBuf.position();
-    }
-    bytesread += len;
-  }
-
-  /**
-   * If a write would exceed the capacity of the direct buffers, it is set
-   * aside to be loaded by this function while the compressed data are
-   * consumed.
-   */
-  synchronized void setInputFromSavedData() {
-    if (0 >= userBufLen) {
-      return;
-    }
-    finished = false;
-
-    uncompressedDirectBufLen = Math.min(userBufLen, directBufferSize);
-    ((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
-  }
-
-  /** {@inheritDoc} */
-  public boolean needsInput() {
-    return !(compressedDirectBuf.remaining() > 0
-        || uncompressedDirectBuf.remaining() == 0
-        || userBufLen > 0);
-  }
-
-  public synchronized void finish() {
-    finish = true;
-  }
-  
-  public synchronized boolean finished() {
-    // Check if 'lzo' says its 'finished' and
-    // all compressed data has been consumed
-    return (finish && 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();
-    }
-
-    // Check if there is compressed data
-    int n = compressedDirectBuf.remaining();
-    if (n > 0) {
-      n = Math.min(n, len);
-      ((ByteBuffer)compressedDirectBuf).get(b, off, n);
-      byteswritten += n;
-      return n;
-    }
-
-    // Re-initialize the lzo's output direct-buffer
-    compressedDirectBuf.clear();
-    compressedDirectBuf.limit(0);
-    if (0 == uncompressedDirectBuf.position()) {
-      // No compressed data, so we should have !needsInput or !finished
-      setInputFromSavedData();
-      if (0 == uncompressedDirectBuf.position()) {
-        // Called without data; write nothing
-        finished = true;
-        return 0;
-      }
-    }
-
-    // Compress data
-    n = compressBytesDirect(strategy.getCompressor());
-    compressedDirectBuf.limit(n);
-    uncompressedDirectBuf.clear(); // lzo consumes all buffer input
-
-    // Set 'finished' if lzo has consumed all user-data
-    if (0 == userBufLen) {
-      finished = true;
-    }
-
-    // Get atmost 'len' bytes
-    n = Math.min(n, len);
-    byteswritten += n;
-    ((ByteBuffer)compressedDirectBuf).get(b, off, n);
-
-    return n;
-  }
-
-  public synchronized void reset() {
-    finish = false;
-    finished = false;
-    uncompressedDirectBuf.clear();
-    uncompressedDirectBufLen = 0;
-    compressedDirectBuf.clear();
-    compressedDirectBuf.limit(0);
-    userBufOff = userBufLen = 0;
-    bytesread = byteswritten = 0L;
-  }
-
-  /**
-   * Return number of bytes given to this compressor since last reset.
-   */
-  public synchronized long getBytesRead() {
-    return bytesread;
-  }
-
-  /**
-   * Return number of bytes consumed by callers of compress since last reset.
-   */
-  public synchronized long getBytesWritten() {
-    return byteswritten;
-  }
-
-  /**
-   * Noop.
-   */
-  public synchronized void end() {
-    // nop
-  }
-  
-  private native static void initIDs();
-  private native static int getLzoLibraryVersion();
-  private native void init(int compressor);
-  private native int compressBytesDirect(int compressor);
-}

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

@@ -1,321 +0,0 @@
-/*
- * 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/
- * 
- */
-public class LzoDecompressor implements Decompressor {
-  private static final Log LOG = 
-    LogFactory.getLog(LzoDecompressor.class.getName());
-
-  // HACK - Use this as a global lock in the JNI layer
-  private static Class clazz = LzoDecompressor.class;
-  
-  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;
-  public static final int LZO_LIBRARY_VERSION;
-  
-  static {
-    if (NativeCodeLoader.isNativeCodeLoaded()) {
-      // Initialize the native library
-      try {
-        initIDs();
-        nativeLzoLoaded = true;
-      } catch (Throwable t) {
-        // Ignore failure to load/initialize native-lzo
-        nativeLzoLoaded = false;
-      }
-      LZO_LIBRARY_VERSION = (nativeLzoLoaded) ? 0xFFFF & getLzoLibraryVersion()
-                                              : -1;
-    } else {
-      LOG.error("Cannot load " + LzoDecompressor.class.getName() + 
-                " without native-hadoop library!");
-      nativeLzoLoaded = false;
-      LZO_LIBRARY_VERSION = -1;
-     }
-   }
-  
-  /**
-   * 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 static int getLzoLibraryVersion();
-  private native void init(int decompressor);
-  private native int decompressBytesDirect(int decompressor);
-}

+ 6 - 6
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -1514,8 +1514,8 @@
         TextInputFormat</a> is the default <code>InputFormat</code>.</p>
         
         <p>If <code>TextInputFormat</code> is the <code>InputFormat</code> for a 
-        given job, the framework detects input-files with the <em>.gz</em> and 
-        <em>.lzo</em> extensions and automatically decompresses them using the 
+        given job, the framework detects input-files with the <em>.gz</em>
+        extensions and automatically decompresses them using the 
         appropriate <code>CompressionCodec</code>. However, it must be noted that
         compressed files with the above extensions cannot be <em>split</em> and 
         each compressed file is processed in its entirety by a single mapper.</p>
@@ -1963,14 +1963,14 @@
           specify compression for both intermediate map-outputs and the
           job-outputs i.e. output of the reduces. It also comes bundled with
           <a href="ext:api/org/apache/hadoop/io/compress/compressioncodec">
-          CompressionCodec</a> implementations for the 
-          <a href="ext:zlib">zlib</a> and <a href="ext:lzo">lzo</a> compression 
-          algorithms. The <a href="ext:gzip">gzip</a> file format is also
+          CompressionCodec</a> implementation for the 
+          <a href="ext:zlib">zlib</a> compression 
+          algorithm. The <a href="ext:gzip">gzip</a> file format is also
           supported.</p>
           
           <p>Hadoop also provides native implementations of the above compression
           codecs for reasons of both performance (zlib) and non-availability of
-          Java libraries (lzo). More details on their usage and availability are
+          Java libraries. More details on their usage and availability are
           available <a href="native_libraries.html">here</a>.</p>
           
           <section>

+ 5 - 9
src/docs/src/documentation/content/xdocs/native_libraries.xml

@@ -44,12 +44,11 @@
       <ul>
         <li><a href="ext:zlib">zlib</a></li>
         <li><a href="ext:gzip">gzip</a></li>
-        <li><a href="ext:lzo">lzo</a></li>
         <li><a href="ext:bzip">bzip2</a></li>
       </ul>
       
       <p>Of the above, the availability of native hadoop libraries is imperative 
-      for the lzo, gzip and bzip2 compression codecs to work.</p>
+      for the gzip and bzip2 compression codecs to work.</p>
     </section>
 
     <section>
@@ -70,8 +69,8 @@
         </li>
         <li>
           Make sure you have any of or all of <strong>&gt;zlib-1.2</strong>,
-          <strong>&gt;gzip-1.2</strong>, <strong>&gt;bzip2-1.0</strong> and 
-          <strong>&gt;lzo2.0</strong> packages for your platform installed; 
+          <strong>&gt;gzip-1.2</strong>, and <strong>&gt;bzip2-1.0</strong>
+          packages for your platform installed; 
           depending on your needs.
         </li>
       </ul>
@@ -147,9 +146,6 @@
         <li> 
           zlib-development package (stable version >= 1.2.0)
         </li>
-        <li> 
-          lzo-development package (stable version >= 2.0)
-        </li> 
       </ul>
 
       <p>Once you have the pre-requisites use the standard <code>build.xml</code> 
@@ -175,13 +171,13 @@
         <ul>
           <li>
             It is <strong>mandatory</strong> to have the 
-            zlib, gzip, bzip2 and lzo 
+            zlib, gzip, and bzip2
             development packages on the target platform for building the 
             native hadoop library; however for deployment it is sufficient to 
             install one of them if you wish to use only one of them.
           </li>
           <li>
-            It is necessary to have the correct 32/64 libraries of both zlib/lzo 
+            It is necessary to have the correct 32/64 libraries of both zlib 
             depending on the 32/64 bit jvm for the target platform for 
             building/deployment of the native hadoop library.
           </li>

+ 0 - 1
src/docs/src/documentation/content/xdocs/site.xml

@@ -73,7 +73,6 @@ See http://forrest.apache.org/docs/linking.html for more info.
     <faq       href="http://wiki.apache.org/hadoop/FAQ" />
     <hadoop-default href="http://hadoop.apache.org/core/docs/current/hadoop-default.html" />
     <zlib      href="http://www.zlib.net/" />
-    <lzo       href="http://www.oberhumer.com/opensource/lzo/" />
     <gzip      href="http://www.gzip.org/" />
     <bzip      href="http://www.bzip.org/" />
     <cygwin    href="http://www.cygwin.com/" />

+ 0 - 385
src/mapred/org/apache/hadoop/mapred/LzoTextInputFormat.java

@@ -1,385 +0,0 @@
-/**
- * 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.mapred;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.io.compress.LzopCodec;
-import org.apache.hadoop.io.compress.LzopCodec.LzopDecompressor;
-import org.apache.hadoop.util.LineReader;
-
-/**
- * An {@link InputFormat} for lzop compressed text files. Files are broken into
- * lines. Either linefeed or carriage-return are used to signal end of line.
- * Keys are the position in the file, and values are the line of text.
- */
-public class LzoTextInputFormat extends FileInputFormat<LongWritable, Text>
-    implements JobConfigurable {
-
-  private static final Log LOG
-    = LogFactory.getLog(LzoTextInputFormat.class.getName());
-  
-  public static final String LZO_INDEX_SUFFIX = ".index";
-
-  public void configure(JobConf conf) {
-    FileInputFormat.setInputPathFilter(conf, LzopFilter.class);
-  }
-
-  /**
-   * We don't want to process the index files.
-   */
-  static class LzopFilter implements PathFilter {
-    public boolean accept(Path path) {
-      if (path.toString().endsWith(LZO_INDEX_SUFFIX)) {
-        return false;
-      }
-      return true;
-    }
-  }
-
-  protected boolean isSplitable(FileSystem fs, Path file) {
-    Path indexFile = new Path(file.toString()
-        + LzoTextInputFormat.LZO_INDEX_SUFFIX);
-
-    try {
-      // can't split without the index
-      return fs.exists(indexFile);
-    } catch (IOException e) {
-      LOG.warn("Could not check if index file exists", e);
-      return false;
-    }
-  }
-
-  public RecordReader<LongWritable, Text> getRecordReader(
-      InputSplit genericSplit, JobConf job, Reporter reporter)
-    throws IOException {
-
-    reporter.setStatus(genericSplit.toString());
-    return new LzoLineRecordReader(job, (FileSplit) genericSplit);
-  }
-
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    FileSplit[] splits = (FileSplit[]) super.getSplits(job, numSplits);
-    // find new start/ends of the filesplit that aligns
-    // with the lzo blocks
-
-    List<FileSplit> result = new ArrayList<FileSplit>();
-    FileSystem fs = FileSystem.get(job);
-
-    Map<Path, LzoIndex> indexes = new HashMap<Path, LzoIndex>();
-    for (int i = 0; i < splits.length; i++) {
-      FileSplit fileSplit = splits[i];
-      // load the index
-      Path file = fileSplit.getPath();
-      if (!indexes.containsKey(file)) {
-        LzoIndex index = readIndex(file, fs);
-        if (index.isEmpty()) {
-          // keep it as is since we didn't find an index
-          result.add(fileSplit);
-          continue;
-        }
-
-        indexes.put(file, index);
-      }
-
-      LzoIndex index = indexes.get(file);
-      long start = fileSplit.getStart();
-      long end = start + fileSplit.getLength();
-
-      if (start != 0) {
-        // find the next block position from
-        // the start of the split
-        long newStart = index.findNextPosition(start);
-        if (newStart == -1 || newStart >= end) {
-          // just skip this since it will be handled by another split
-          continue;
-        }
-        start = newStart;
-      }
-
-      long newEnd = index.findNextPosition(end);
-      if (newEnd != -1) {
-        end = newEnd;
-      }
-
-      result.add(new FileSplit(file, start, end - start, fileSplit
-          .getLocations()));
-    }
-
-    return result.toArray(new FileSplit[] {});
-  }
-
-  /**
-   * Read the index of the lzo file.
-   * 
-   * @param split Read the index of this file.
-   * @param fs The index file is on this file system.
-   * @throws IOException
-   */
-  private LzoIndex readIndex(Path file, FileSystem fs) throws IOException {
-    FSDataInputStream indexIn = null;
-    try {
-      Path indexFile = new Path(file.toString() + LZO_INDEX_SUFFIX);
-      if (!fs.exists(indexFile)) {
-        // return empty index, fall back to the unsplittable mode
-        return new LzoIndex();
-      }
-      
-      long indexLen = fs.getFileStatus(indexFile).getLen();
-      int blocks = (int) (indexLen / 8);
-      LzoIndex index = new LzoIndex(blocks);
-      indexIn = fs.open(indexFile);
-      for (int i = 0; i < blocks; i++) {
-        index.set(i, indexIn.readLong());
-      }
-      return index;
-    } finally {
-      if (indexIn != null) {
-        indexIn.close();
-      }
-    }
-  }
-
-  /**
-   * Index an lzo file to allow the input format to split them into separate map
-   * jobs.
-   * 
-   * @param fs File system that contains the file.
-   * @param lzoFile the lzo file to index.
-   * @throws IOException
-   */
-  public static void createIndex(FileSystem fs, Path lzoFile) 
-    throws IOException {
-    
-    Configuration conf = fs.getConf();
-    LzopCodec codec = new LzopCodec();
-    codec.setConf(conf);
-
-    FSDataInputStream is = null;
-    FSDataOutputStream os = null;
-    try {
-      is = fs.open(lzoFile);
-      os = fs.create(new Path(lzoFile.toString()
-          + LzoTextInputFormat.LZO_INDEX_SUFFIX));
-      LzopDecompressor decompressor = (LzopDecompressor) codec
-          .createDecompressor();
-      // for reading the header
-      codec.createInputStream(is, decompressor);
-
-      int numChecksums = decompressor.getChecksumsCount();
-
-      while (true) {
-        //read and ignore, we just want to get to the next int
-        int uncompressedBlockSize = is.readInt();
-        if (uncompressedBlockSize == 0) {
-          break;
-        } else if (uncompressedBlockSize < 0) {
-          throw new EOFException();
-        }
-        
-        int compressedBlockSize = is.readInt();
-        if (compressedBlockSize <= 0) {
-          throw new IOException("Could not read compressed block size");
-        }
-
-        long pos = is.getPos();
-        // write the pos of the block start
-        os.writeLong(pos - 8);
-        // seek to the start of the next block, skip any checksums
-        is.seek(pos + compressedBlockSize + (4 * numChecksums));
-      }
-    } finally {
-      if (is != null) {
-        is.close();
-      }
-
-      if (os != null) {
-        os.close();
-      }
-    }
-  }
-
-  /**
-   * Represents the lzo index.
-   */
-  static class LzoIndex {
-    
-    private long[] blockPositions;
-
-    LzoIndex() {
-    }   
-    
-    LzoIndex(int blocks) {
-      blockPositions = new long[blocks];
-    }
-    
-    /**
-     * Set the position for the block.
-     * @param blockNumber Block to set pos for.
-     * @param pos Position.
-     */
-    public void set(int blockNumber, long pos) {
-      blockPositions[blockNumber] = pos;
-    }
-    
-    /**
-     * Find the next lzo block start from the given position.
-     * @param pos The position to start looking from.
-     * @return Either the start position of the block or -1 if 
-     * it couldn't be found.
-     */
-    public long findNextPosition(long pos) {
-      int block = Arrays.binarySearch(blockPositions, pos);
-
-      if(block >= 0) {
-        //direct hit on a block start position
-        return blockPositions[block];
-      } else {
-        block = Math.abs(block) - 1;
-        if(block > blockPositions.length - 1) {
-          return -1;
-        }
-        return blockPositions[block];
-      }
-    }
-
-    public boolean isEmpty() {
-      return blockPositions == null || blockPositions.length == 0;
-    }    
-    
-  }
-  
-  /**
-   * Reads line from an lzo compressed text file. Treats keys as offset in file
-   * and value as line.
-   */
-  static class LzoLineRecordReader implements RecordReader<LongWritable, Text> {
-
-    private CompressionCodecFactory compressionCodecs = null;
-    private long start;
-    private long pos;
-    private long end;
-    private LineReader in;
-    private FSDataInputStream fileIn;
-
-    public LzoLineRecordReader(Configuration job, FileSplit split)
-      throws IOException {
-
-      start = split.getStart();
-      end = start + split.getLength();
-      final Path file = split.getPath();
-
-      FileSystem fs = file.getFileSystem(job);
-
-      compressionCodecs = new CompressionCodecFactory(job);
-      final CompressionCodec codec = compressionCodecs.getCodec(file);
-      if (codec == null) {
-        throw new IOException("No lzo codec found, cannot run");
-      }
-
-      // open the file and seek to the start of the split
-      fileIn = fs.open(split.getPath());
-
-      // creates input stream and also reads the file header
-      in = new LineReader(codec.createInputStream(fileIn), job);
-
-      if (start != 0) {
-        fileIn.seek(start);
-
-        // read and ignore the first line
-        in.readLine(new Text());
-        start = fileIn.getPos();
-      }
-
-      this.pos = start;
-    }
-
-    public LongWritable createKey() {
-      return new LongWritable();
-    }
-
-    public Text createValue() {
-      return new Text();
-    }
-
-    /** Read a line. */
-    public synchronized boolean next(LongWritable key, Text value)
-      throws IOException {
-
-      //since the lzop codec reads everything in lzo blocks
-      //we can't stop if the pos == end
-      //instead we wait for the next block to be read in when
-      //pos will be > end
-      while (pos <= end) {
-        key.set(pos);
-
-        int newSize = in.readLine(value);
-        if (newSize == 0) {
-          return false;
-        }
-        pos = fileIn.getPos();
-
-        return true;
-      }
-
-      return false;
-    }
-
-    /**
-     * Get the progress within the split.
-     */
-    public float getProgress() {
-      if (start == end) {
-        return 0.0f;
-      } else {
-        return Math.min(1.0f, (pos - start) / (float) (end - start));
-      }
-    }
-
-    public synchronized long getPos() throws IOException {
-      return pos;
-    }
-
-    public synchronized void close() throws IOException {
-      if (in != null) {
-        in.close();
-      }
-    }
-  }
-
-}

+ 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 src/org/apache/hadoop/io/compress/lzo lib
+SUBDIRS = src/org/apache/hadoop/io/compress/zlib
 
 # The following export is needed to build libhadoop.so in the 'lib' directory
 export SUBDIRS

+ 7 - 20
src/native/Makefile.in

@@ -1,8 +1,8 @@
-# Makefile.in generated by automake 1.9.6 from Makefile.am.
+# Makefile.in generated by automake 1.9 from Makefile.am.
 # @configure_input@
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
-# 2003, 2004, 2005  Free Software Foundation, Inc.
+# 2003, 2004  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.
@@ -67,7 +67,6 @@ POST_INSTALL = :
 NORMAL_UNINSTALL = :
 PRE_UNINSTALL = :
 POST_UNINSTALL = :
-build_triplet = @build@
 host_triplet = @host@
 DIST_COMMON = README $(am__configure_deps) $(srcdir)/Makefile.am \
 	$(srcdir)/Makefile.in $(srcdir)/config.h.in \
@@ -207,7 +206,7 @@ sysconfdir = @sysconfdir@
 target_alias = @target_alias@
 
 # List the sub-directories here
-SUBDIRS = src/org/apache/hadoop/io/compress/zlib src/org/apache/hadoop/io/compress/lzo lib
+SUBDIRS = src/org/apache/hadoop/io/compress/zlib
 all: config.h
 	$(MAKE) $(AM_MAKEFLAGS) all-recursive
 
@@ -280,13 +279,7 @@ uninstall-info-am:
 #     (which will cause the Makefiles to be regenerated when you run `make');
 # (2) otherwise, pass the desired values on the `make' command line.
 $(RECURSIVE_TARGETS):
-	@failcom='exit 1'; \
-	for f in x $$MAKEFLAGS; do \
-	  case $$f in \
-	    *=* | --[!k]*);; \
-	    *k*) failcom='fail=yes';; \
-	  esac; \
-	done; \
+	@set fnord $$MAKEFLAGS; amf=$$2; \
 	dot_seen=no; \
 	target=`echo $@ | sed s/-recursive//`; \
 	list='$(SUBDIRS)'; for subdir in $$list; do \
@@ -298,7 +291,7 @@ $(RECURSIVE_TARGETS):
 	    local_target="$$target"; \
 	  fi; \
 	  (cd $$subdir && $(MAKE) $(AM_MAKEFLAGS) $$local_target) \
-	  || eval $$failcom; \
+	   || case "$$amf" in *=*) exit 1;; *k*) fail=yes;; *) exit 1;; esac; \
 	done; \
 	if test "$$dot_seen" = "no"; then \
 	  $(MAKE) $(AM_MAKEFLAGS) "$$target-am" || exit 1; \
@@ -306,13 +299,7 @@ $(RECURSIVE_TARGETS):
 
 mostlyclean-recursive clean-recursive distclean-recursive \
 maintainer-clean-recursive:
-	@failcom='exit 1'; \
-	for f in x $$MAKEFLAGS; do \
-	  case $$f in \
-	    *=* | --[!k]*);; \
-	    *k*) failcom='fail=yes';; \
-	  esac; \
-	done; \
+	@set fnord $$MAKEFLAGS; amf=$$2; \
 	dot_seen=no; \
 	case "$@" in \
 	  distclean-* | maintainer-clean-*) list='$(DIST_SUBDIRS)' ;; \
@@ -333,7 +320,7 @@ maintainer-clean-recursive:
 	    local_target="$$target"; \
 	  fi; \
 	  (cd $$subdir && $(MAKE) $(AM_MAKEFLAGS) $$local_target) \
-	  || eval $$failcom; \
+	   || case "$$amf" in *=*) exit 1;; *k*) fail=yes;; *) exit 1;; esac; \
 	done && test -z "$$fail"
 tags-recursive:
 	list='$(SUBDIRS)'; for subdir in $$list; do \

文件差异内容过多而无法显示
+ 151 - 356
src/native/aclocal.m4


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

@@ -1,8 +1,5 @@
 /* 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
 
@@ -21,36 +18,6 @@
 /* Define to 1 if you have the `jvm' library (-ljvm). */
 #undef HAVE_LIBJVM
 
-/* 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
 

文件差异内容过多而无法显示
+ 193 - 401
src/native/configure


+ 0 - 4
src/native/configure.ac

@@ -86,9 +86,6 @@ 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,7 +94,6 @@ 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
 

+ 6 - 5
src/native/lib/Makefile.in

@@ -1,8 +1,8 @@
-# Makefile.in generated by automake 1.9.6 from Makefile.am.
+# Makefile.in generated by automake 1.9 from Makefile.am.
 # @configure_input@
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
-# 2003, 2004, 2005  Free Software Foundation, Inc.
+# 2003, 2004  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.
@@ -46,6 +46,8 @@
 #    All these are setup by build.xml and/or the top-level makefile.
 #
 
+SOURCES = $(libhadoop_la_SOURCES)
+
 srcdir = @srcdir@
 top_srcdir = @top_srcdir@
 VPATH = @srcdir@
@@ -66,7 +68,6 @@ POST_INSTALL = :
 NORMAL_UNINSTALL = :
 PRE_UNINSTALL = :
 POST_UNINSTALL = :
-build_triplet = @build@
 host_triplet = @host@
 subdir = lib
 DIST_COMMON = $(srcdir)/Makefile.am $(srcdir)/Makefile.in
@@ -95,11 +96,11 @@ libhadoop_la_OBJECTS = $(am_libhadoop_la_OBJECTS)
 DEFAULT_INCLUDES = -I. -I$(srcdir) -I$(top_builddir)
 COMPILE = $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) \
 	$(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS)
-LTCOMPILE = $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) \
+LTCOMPILE = $(LIBTOOL) --mode=compile --tag=CC $(CC) $(DEFS) \
 	$(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) \
 	$(AM_CFLAGS) $(CFLAGS)
 CCLD = $(CC)
-LINK = $(LIBTOOL) --tag=CC --mode=link $(CCLD) $(AM_CFLAGS) $(CFLAGS) \
+LINK = $(LIBTOOL) --mode=link --tag=CC $(CCLD) $(AM_CFLAGS) $(CFLAGS) \
 	$(AM_LDFLAGS) $(LDFLAGS) -o $@
 SOURCES = $(libhadoop_la_SOURCES)
 DIST_SOURCES = $(libhadoop_la_SOURCES)

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

@@ -1,301 +0,0 @@
-/**
- * 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 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;
-// lzo2 library version
-static jint liblzo2_version = 0;
-
-// 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_clazz;
-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_clazz = (*env)->GetStaticFieldID(env, class, "clazz", 
-                                                 "Ljava/lang/Class;");
-  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;");
-
-  // record lzo library version
-  void* lzo_version_ptr = NULL;
-  LOAD_DYNAMIC_SYMBOL(lzo_version_ptr, env, liblzo2, "lzo_version");
-  liblzo2_version = (NULL == lzo_version_ptr) ? 0
-    : (jint) ((unsigned (__LZO_CDECL *)())lzo_version_ptr)();
-}
-
-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_getLzoLibraryVersion(
-    JNIEnv* env, jclass class) {
-  return liblzo2_version;
-}
-
-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 clazz = (*env)->GetStaticObjectField(env, this, 
-                                                 LzoCompressor_clazz);
-	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 the input direct buffer
-    LOCK_CLASS(env, clazz, "LzoCompressor");
-	lzo_bytep uncompressed_bytes = (*env)->GetDirectBufferAddress(env, 
-                                            uncompressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "LzoCompressor");
-    
-  if (uncompressed_bytes == 0) {
-    	return (jint)0;
-	}
-	
-    // Get the output direct buffer
-    LOCK_CLASS(env, clazz, "LzoCompressor");
-	lzo_bytep compressed_bytes = (*env)->GetDirectBufferAddress(env, 
-                                            compressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "LzoCompressor");
-    
-  if (compressed_bytes == 0) {
-		return (jint)0;
-	}
-	
-    // Get the working-memory direct buffer
-    LOCK_CLASS(env, clazz, "LzoCompressor");
-    lzo_voidp workmem = (*env)->GetDirectBufferAddress(env, working_memory_buf);
-    UNLOCK_CLASS(env, clazz, "LzoCompressor");
-    
-  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:
- */
-

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

@@ -1,238 +0,0 @@
-/**
- * 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 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;
-// lzo2 library version
-static jint liblzo2_version = 0;
-
-// 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_clazz;
-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_clazz = (*env)->GetStaticFieldID(env, class, "clazz", 
-                                                   "Ljava/lang/Class;");
-  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");
-
-  // record lzo library version
-  void* lzo_version_ptr = NULL;
-  LOAD_DYNAMIC_SYMBOL(lzo_version_ptr, env, liblzo2, "lzo_version");
-  liblzo2_version = (NULL == lzo_version_ptr) ? 0
-    : (jint) ((unsigned (__LZO_CDECL *)())lzo_version_ptr)();
-}
-
-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_getLzoLibraryVersion(
-    JNIEnv* env, jclass class) {
-  return liblzo2_version;
-}
-
-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 clazz = (*env)->GetStaticObjectField(env, this, 
-	                                             LzoDecompressor_clazz);
-	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 the input direct buffer
-    LOCK_CLASS(env, clazz, "LzoDecompressor");
-	lzo_bytep uncompressed_bytes = (*env)->GetDirectBufferAddress(env, 
-											                    uncompressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "LzoDecompressor");
-    
- 	if (uncompressed_bytes == 0) {
-    return (jint)0;
-	}
-	
-    // Get the output direct buffer
-    LOCK_CLASS(env, clazz, "LzoDecompressor");
-	lzo_bytep compressed_bytes = (*env)->GetDirectBufferAddress(env, 
-										                    compressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "LzoDecompressor");
-
-  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:
- */
-

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

@@ -1,50 +0,0 @@
-#
-# 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
-#

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

@@ -1,469 +0,0 @@
-# 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:

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

@@ -1,112 +0,0 @@
-/**
- * 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

+ 6 - 5
src/native/src/org/apache/hadoop/io/compress/zlib/Makefile.in

@@ -1,8 +1,8 @@
-# Makefile.in generated by automake 1.9.6 from Makefile.am.
+# Makefile.in generated by automake 1.9 from Makefile.am.
 # @configure_input@
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
-# 2003, 2004, 2005  Free Software Foundation, Inc.
+# 2003, 2004  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.
@@ -50,6 +50,8 @@
 #    assumed to be in $(HADOOP_HOME)/build/native/src/org/apache/hadoop/io/compress/zlib.
 #
 
+SOURCES = $(libnativezlib_la_SOURCES)
+
 srcdir = @srcdir@
 top_srcdir = @top_srcdir@
 VPATH = @srcdir@
@@ -70,7 +72,6 @@ POST_INSTALL = :
 NORMAL_UNINSTALL = :
 PRE_UNINSTALL = :
 POST_UNINSTALL = :
-build_triplet = @build@
 host_triplet = @host@
 subdir = src/org/apache/hadoop/io/compress/zlib
 DIST_COMMON = $(srcdir)/Makefile.am $(srcdir)/Makefile.in
@@ -91,11 +92,11 @@ 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) \
+LTCOMPILE = $(LIBTOOL) --mode=compile --tag=CC $(CC) $(DEFS) \
 	$(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) \
 	$(AM_CFLAGS) $(CFLAGS)
 CCLD = $(CC)
-LINK = $(LIBTOOL) --tag=CC --mode=link $(CCLD) $(AM_CFLAGS) $(CFLAGS) \
+LINK = $(LIBTOOL) --mode=link --tag=CC $(CCLD) $(AM_CFLAGS) $(CFLAGS) \
 	$(AM_LDFLAGS) $(LDFLAGS) -o $@
 SOURCES = $(libnativezlib_la_SOURCES)
 DIST_SOURCES = $(libnativezlib_la_SOURCES)

+ 4 - 5
src/test/org/apache/hadoop/io/FileBench.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.LzoCodec;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.*;
@@ -46,12 +45,12 @@ public class FileBench extends Configured implements Tool {
     System.out.println(
 "Usage: Task list:           -[no]r -[no]w\n" +
 "       Format:              -[no]seq -[no]txt\n" +
-"       CompressionCodec:    -[no]lzo -[no]zip -[no]pln\n" +
+"       CompressionCodec:    -[no]zip -[no]pln\n" +
 "       CompressionType:     -[no]blk -[no]rec\n" +
 "       Required:            -dir <working dir>\n" +
 "All valid combinations are implicitly enabled, unless an option is enabled\n" +
-"explicitly. For example, specifying \"-lzo\" excludes -zip and -pln,\n" +
-"unless they are also explicitly included, as in \"-lzo -zip\"\n" +
+"explicitly. For example, specifying \"-zip\", excludes -pln,\n" +
+"unless they are also explicitly included, as in \"-pln -zip\"\n" +
 "Note that CompressionType params only apply to SequenceFiles\n\n" +
 "Useful options to set:\n" +
 "-D fs.default.name=\"file:///\" \\\n" +
@@ -243,7 +242,7 @@ public class FileBench extends Configured implements Tool {
 
   // overwrought argument processing and wordlist follow
   enum CCodec {
-    lzo(LzoCodec.class, ".lzo"), zip(GzipCodec.class, ".gz"), pln(null, "");
+    zip(GzipCodec.class, ".gz"), pln(null, "");
 
     Class<? extends CompressionCodec> inf;
     String ext;

+ 0 - 17
src/test/org/apache/hadoop/io/TestSequenceFile.java

@@ -28,7 +28,6 @@ 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.*;
 
@@ -48,22 +47,6 @@ public class TestSequenceFile extends TestCase {
     LOG.info("Successfully tested SequenceFile with DefaultCodec");
   }
   
-  public void testLzoSequenceFile() throws Exception {
-    if (LzoCodec.isNativeLzoLoaded(conf)) {
-      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 {
     int count = 1024 * 10;
     int megabytes = 1;

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

@@ -52,18 +52,6 @@ public class TestCodec extends TestCase {
     codecTest(conf, seed, count, "org.apache.hadoop.io.compress.GzipCodec");
   }
   
-  public void testLzoCodec() throws IOException {
-    if (LzoCodec.isNativeLzoLoaded(conf)) {
-      codecTest(conf, seed, count, "org.apache.hadoop.io.compress.LzoCodec");
-    }
-  }
-  
-  public void testLzopCodec() throws IOException {
-    if (LzopCodec.isNativeLzoLoaded(conf)) {
-      codecTest(conf, seed, count, "org.apache.hadoop.io.compress.LzopCodec");
-    }
-  }
-  
   public void testBZip2Codec() throws IOException {    
       codecTest(conf, seed, count, "org.apache.hadoop.io.compress.BZip2Codec");    
   }

+ 0 - 222
src/test/org/apache/hadoop/mapred/TestLzoTextInputFormat.java

@@ -1,222 +0,0 @@
-/**
- * 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.mapred;
-
-import java.io.IOException;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-import java.util.Random;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.LzopCodec;
-import org.apache.hadoop.mapred.LzoTextInputFormat.LzoIndex;
-import org.apache.hadoop.util.NativeCodeLoader;
-
-/**
- * Test the LzoTextInputFormat, make sure it splits the file properly and
- * returns the right data.
- */
-public class TestLzoTextInputFormat extends TestCase {
-
-  private static final Log LOG
-    = LogFactory.getLog(TestLzoTextInputFormat.class.getName());
-  
-  private MessageDigest md5;
-  private String lzoFileName = "file";
-  
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    md5 = MessageDigest.getInstance("MD5");
-  }
-  
-  /**
-   * Make sure the lzo index class works as described.
-   */
-  public void testLzoIndex() {
-    LzoIndex index = new LzoIndex();
-    assertTrue(index.isEmpty());
-    index = new LzoIndex(4);
-    index.set(0, 0);
-    index.set(1, 5);
-    index.set(2, 10);
-    index.set(3, 15);
-    assertFalse(index.isEmpty());
-    
-    assertEquals(0, index.findNextPosition(-1));
-    assertEquals(5, index.findNextPosition(1));
-    assertEquals(5, index.findNextPosition(5));
-    assertEquals(15, index.findNextPosition(11));
-    assertEquals(15, index.findNextPosition(15));
-    assertEquals(-1, index.findNextPosition(16));
-  }
-  
-  /**
-   * Index the file and make sure it splits properly.
-   * @throws NoSuchAlgorithmException
-   * @throws IOException
-   */
-  public void testWithIndex() throws NoSuchAlgorithmException, IOException {
-    runTest(true);
-  }
-  
-  /**
-   * Don't index the file and make sure it can be processed anyway.
-   * @throws NoSuchAlgorithmException
-   * @throws IOException
-   */
-  public void testWithoutIndex() throws NoSuchAlgorithmException, IOException {
-    runTest(false);
-  }
-  
-  private void runTest(boolean testWithIndex) throws IOException, 
-    NoSuchAlgorithmException {
-    
-    if (!NativeCodeLoader.isNativeCodeLoaded()) {
-      LOG.warn("Cannot run this test without the native lzo libraries");
-      return;
-    }
-
-    String attempt = "attempt_200707121733_0001_m_000000_0";
-    Path workDir = new Path(new Path(new Path(System.getProperty(
-        "test.build.data", "."), "data"), FileOutputCommitter.TEMP_DIR_NAME),
-        "_" + attempt);
-    Path outputDir = workDir.getParent().getParent();
-
-    JobConf conf = new JobConf();
-    conf.set("mapred.task.id", attempt);
-    conf.set("io.compression.codecs", LzopCodec.class.getName());
-
-    FileSystem localFs = FileSystem.getLocal(conf);
-    localFs.delete(workDir, true);
-    localFs.mkdirs(workDir);
-    FileInputFormat.setInputPaths(conf, outputDir);
-
-    
-    // create some input data
-    byte[] expectedMd5 = createTestInput(outputDir, workDir, conf, localFs);
-
-    if(testWithIndex) {
-      Path lzoFile = new Path(workDir, lzoFileName);
-      LzoTextInputFormat.createIndex(localFs, new Path(lzoFile
-        + new LzopCodec().getDefaultExtension()));
-    }
-    
-    LzoTextInputFormat inputFormat = new LzoTextInputFormat();
-    inputFormat.configure(conf);
-    
-    //it's left in the work dir
-    FileInputFormat.setInputPaths(conf, workDir);
-
-    int numSplits = 3;
-    InputSplit[] is = inputFormat.getSplits(conf, numSplits);
-    if(testWithIndex) {
-      assertEquals(numSplits, is.length);
-    } else {
-      assertEquals(1, is.length);
-    }
-
-    for (InputSplit inputSplit : is) {
-      RecordReader<LongWritable, Text> rr = inputFormat.getRecordReader(
-          inputSplit, conf, Reporter.NULL);
-      LongWritable key = rr.createKey();
-      Text value = rr.createValue();
-
-      while (rr.next(key, value)) {
-        md5.update(value.getBytes(), 0, value.getLength());
-      }
-
-      rr.close();
-    }
-
-    assertTrue(Arrays.equals(expectedMd5, md5.digest()));
-  }
-
-  /**
-   * Creates an lzo file with random data. 
-   * 
-   * @param outputDir Output directory
-   * @param workDir Work directory, this is where the file is written to
-   * @param fs File system we're using
-   * @throws IOException
-   */
-  private byte[] createTestInput(Path outputDir, Path workDir, JobConf conf,
-      FileSystem fs) throws IOException {
-
-    RecordWriter<Text, Text> rw = null;
-    
-    md5.reset();
-    
-    try {
-      TextOutputFormat<Text, Text> output = new TextOutputFormat<Text, Text>();
-      TextOutputFormat.setCompressOutput(conf, true);
-      TextOutputFormat.setOutputCompressorClass(conf, LzopCodec.class);
-      TextOutputFormat.setOutputPath(conf, outputDir);
-      TextOutputFormat.setWorkOutputPath(conf, workDir);
-
-      rw = output.getRecordWriter(null, conf, lzoFileName, Reporter.NULL);
-
-      //has to be enough data to create a couple of lzo blocks
-      int charsToOutput = 10485760;
-      char[] chars = "abcdefghijklmnopqrstuvwxyz\u00E5\u00E4\u00F6"
-          .toCharArray();
-
-      Random r = new Random(System.currentTimeMillis());
-      Text key = new Text();
-      Text value = new Text();
-      int charsMax = chars.length - 1;
-      for (int i = 0; i < charsToOutput;) {
-        i += fillText(chars, r, charsMax, key);
-        i += fillText(chars, r, charsMax, value);
-        rw.write(key, value);
-        md5.update(key.getBytes(), 0, key.getLength());
-        //text output format writes tab between the key and value
-        md5.update("\t".getBytes("UTF-8")); 
-        md5.update(value.getBytes(), 0, value.getLength());
-      }
-    } finally {
-      if (rw != null) {
-        rw.close(Reporter.NULL);
-      }
-    }
-
-    byte[] result = md5.digest();
-    md5.reset();
-    return result;
-  }
-
-  private int fillText(char[] chars, Random r, int charsMax, Text text) {
-    StringBuilder sb = new StringBuilder();
-    //get a reasonable string length 
-    int stringLength = r.nextInt(charsMax * 2);
-    for (int j = 0; j < stringLength; j++) {
-      sb.append(chars[r.nextInt(charsMax)]);
-    }
-    text.set(sb.toString());
-    return stringLength;
-  }
-
-}

部分文件因为文件数量过多而无法显示