浏览代码

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. 
     pipes, utils, and libhdfs are now all in c++/<os_osarch_jvmdatamodel>/lib. 
     (Giridharan Kesavan via nigel)
     (Giridharan Kesavan via nigel)
 
 
+    HADOOP-4874. Remove LZO codec because of licensing issues. (omalley)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-4575. Add a proxy service for relaying HsftpFileSystem requests.
     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>
         TextInputFormat</a> is the default <code>InputFormat</code>.</p>
         
         
         <p>If <code>TextInputFormat</code> is the <code>InputFormat</code> for a 
         <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
         appropriate <code>CompressionCodec</code>. However, it must be noted that
         compressed files with the above extensions cannot be <em>split</em> and 
         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>
         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
           specify compression for both intermediate map-outputs and the
           job-outputs i.e. output of the reduces. It also comes bundled with
           job-outputs i.e. output of the reduces. It also comes bundled with
           <a href="ext:api/org/apache/hadoop/io/compress/compressioncodec">
           <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>
           supported.</p>
           
           
           <p>Hadoop also provides native implementations of the above compression
           <p>Hadoop also provides native implementations of the above compression
           codecs for reasons of both performance (zlib) and non-availability of
           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>
           available <a href="native_libraries.html">here</a>.</p>
           
           
           <section>
           <section>

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

@@ -44,12 +44,11 @@
       <ul>
       <ul>
         <li><a href="ext:zlib">zlib</a></li>
         <li><a href="ext:zlib">zlib</a></li>
         <li><a href="ext:gzip">gzip</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>
         <li><a href="ext:bzip">bzip2</a></li>
       </ul>
       </ul>
       
       
       <p>Of the above, the availability of native hadoop libraries is imperative 
       <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>
 
 
     <section>
     <section>
@@ -70,8 +69,8 @@
         </li>
         </li>
         <li>
         <li>
           Make sure you have any of or all of <strong>&gt;zlib-1.2</strong>,
           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.
           depending on your needs.
         </li>
         </li>
       </ul>
       </ul>
@@ -147,9 +146,6 @@
         <li> 
         <li> 
           zlib-development package (stable version >= 1.2.0)
           zlib-development package (stable version >= 1.2.0)
         </li>
         </li>
-        <li> 
-          lzo-development package (stable version >= 2.0)
-        </li> 
       </ul>
       </ul>
 
 
       <p>Once you have the pre-requisites use the standard <code>build.xml</code> 
       <p>Once you have the pre-requisites use the standard <code>build.xml</code> 
@@ -175,13 +171,13 @@
         <ul>
         <ul>
           <li>
           <li>
             It is <strong>mandatory</strong> to have the 
             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 
             development packages on the target platform for building the 
             native hadoop library; however for deployment it is sufficient to 
             native hadoop library; however for deployment it is sufficient to 
             install one of them if you wish to use only one of them.
             install one of them if you wish to use only one of them.
           </li>
           </li>
           <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 
             depending on the 32/64 bit jvm for the target platform for 
             building/deployment of the native hadoop library.
             building/deployment of the native hadoop library.
           </li>
           </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" />
     <faq       href="http://wiki.apache.org/hadoop/FAQ" />
     <hadoop-default href="http://hadoop.apache.org/core/docs/current/hadoop-default.html" />
     <hadoop-default href="http://hadoop.apache.org/core/docs/current/hadoop-default.html" />
     <zlib      href="http://www.zlib.net/" />
     <zlib      href="http://www.zlib.net/" />
-    <lzo       href="http://www.oberhumer.com/opensource/lzo/" />
     <gzip      href="http://www.gzip.org/" />
     <gzip      href="http://www.gzip.org/" />
     <bzip      href="http://www.bzip.org/" />
     <bzip      href="http://www.bzip.org/" />
     <cygwin    href="http://www.cygwin.com/" />
     <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])
 export PLATFORM = $(shell echo $$OS_NAME | tr [A-Z] [a-z])
 
 
 # List the sub-directories here
 # 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
 # The following export is needed to build libhadoop.so in the 'lib' directory
 export SUBDIRS
 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@
 # @configure_input@
 
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
 # 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
 # This Makefile.in is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
 # with or without modifications, as long as this notice is preserved.
@@ -67,7 +67,6 @@ POST_INSTALL = :
 NORMAL_UNINSTALL = :
 NORMAL_UNINSTALL = :
 PRE_UNINSTALL = :
 PRE_UNINSTALL = :
 POST_UNINSTALL = :
 POST_UNINSTALL = :
-build_triplet = @build@
 host_triplet = @host@
 host_triplet = @host@
 DIST_COMMON = README $(am__configure_deps) $(srcdir)/Makefile.am \
 DIST_COMMON = README $(am__configure_deps) $(srcdir)/Makefile.am \
 	$(srcdir)/Makefile.in $(srcdir)/config.h.in \
 	$(srcdir)/Makefile.in $(srcdir)/config.h.in \
@@ -207,7 +206,7 @@ sysconfdir = @sysconfdir@
 target_alias = @target_alias@
 target_alias = @target_alias@
 
 
 # List the sub-directories here
 # 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
 all: config.h
 	$(MAKE) $(AM_MAKEFLAGS) all-recursive
 	$(MAKE) $(AM_MAKEFLAGS) all-recursive
 
 
@@ -280,13 +279,7 @@ uninstall-info-am:
 #     (which will cause the Makefiles to be regenerated when you run `make');
 #     (which will cause the Makefiles to be regenerated when you run `make');
 # (2) otherwise, pass the desired values on the `make' command line.
 # (2) otherwise, pass the desired values on the `make' command line.
 $(RECURSIVE_TARGETS):
 $(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; \
 	dot_seen=no; \
 	target=`echo $@ | sed s/-recursive//`; \
 	target=`echo $@ | sed s/-recursive//`; \
 	list='$(SUBDIRS)'; for subdir in $$list; do \
 	list='$(SUBDIRS)'; for subdir in $$list; do \
@@ -298,7 +291,7 @@ $(RECURSIVE_TARGETS):
 	    local_target="$$target"; \
 	    local_target="$$target"; \
 	  fi; \
 	  fi; \
 	  (cd $$subdir && $(MAKE) $(AM_MAKEFLAGS) $$local_target) \
 	  (cd $$subdir && $(MAKE) $(AM_MAKEFLAGS) $$local_target) \
-	  || eval $$failcom; \
+	   || case "$$amf" in *=*) exit 1;; *k*) fail=yes;; *) exit 1;; esac; \
 	done; \
 	done; \
 	if test "$$dot_seen" = "no"; then \
 	if test "$$dot_seen" = "no"; then \
 	  $(MAKE) $(AM_MAKEFLAGS) "$$target-am" || exit 1; \
 	  $(MAKE) $(AM_MAKEFLAGS) "$$target-am" || exit 1; \
@@ -306,13 +299,7 @@ $(RECURSIVE_TARGETS):
 
 
 mostlyclean-recursive clean-recursive distclean-recursive \
 mostlyclean-recursive clean-recursive distclean-recursive \
 maintainer-clean-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; \
 	dot_seen=no; \
 	case "$@" in \
 	case "$@" in \
 	  distclean-* | maintainer-clean-*) list='$(DIST_SUBDIRS)' ;; \
 	  distclean-* | maintainer-clean-*) list='$(DIST_SUBDIRS)' ;; \
@@ -333,7 +320,7 @@ maintainer-clean-recursive:
 	    local_target="$$target"; \
 	    local_target="$$target"; \
 	  fi; \
 	  fi; \
 	  (cd $$subdir && $(MAKE) $(AM_MAKEFLAGS) $$local_target) \
 	  (cd $$subdir && $(MAKE) $(AM_MAKEFLAGS) $$local_target) \
-	  || eval $$failcom; \
+	   || case "$$amf" in *=*) exit 1;; *k*) fail=yes;; *) exit 1;; esac; \
 	done && test -z "$$fail"
 	done && test -z "$$fail"
 tags-recursive:
 tags-recursive:
 	list='$(SUBDIRS)'; for subdir in $$list; do \
 	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.  */
 /* 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' */
 /* The 'actual' dynamic-library for '-lz' */
 #undef HADOOP_ZLIB_LIBRARY
 #undef HADOOP_ZLIB_LIBRARY
 
 
@@ -21,36 +18,6 @@
 /* Define to 1 if you have the `jvm' library (-ljvm). */
 /* Define to 1 if you have the `jvm' library (-ljvm). */
 #undef HAVE_LIBJVM
 #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. */
 /* Define to 1 if you have the <memory.h> header file. */
 #undef HAVE_MEMORY_H
 #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
 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.))
 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.
 # Checks for typedefs, structures, and compiler characteristics.
 AC_C_CONST
 AC_C_CONST
 
 
@@ -97,7 +94,6 @@ AC_CHECK_FUNCS([memset])
 
 
 AC_CONFIG_FILES([Makefile
 AC_CONFIG_FILES([Makefile
                  src/org/apache/hadoop/io/compress/zlib/Makefile
                  src/org/apache/hadoop/io/compress/zlib/Makefile
-                 src/org/apache/hadoop/io/compress/lzo/Makefile
                  lib/Makefile])
                  lib/Makefile])
 AC_OUTPUT
 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@
 # @configure_input@
 
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
 # 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
 # This Makefile.in is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
 # 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.
 #    All these are setup by build.xml and/or the top-level makefile.
 #
 #
 
 
+SOURCES = $(libhadoop_la_SOURCES)
+
 srcdir = @srcdir@
 srcdir = @srcdir@
 top_srcdir = @top_srcdir@
 top_srcdir = @top_srcdir@
 VPATH = @srcdir@
 VPATH = @srcdir@
@@ -66,7 +68,6 @@ POST_INSTALL = :
 NORMAL_UNINSTALL = :
 NORMAL_UNINSTALL = :
 PRE_UNINSTALL = :
 PRE_UNINSTALL = :
 POST_UNINSTALL = :
 POST_UNINSTALL = :
-build_triplet = @build@
 host_triplet = @host@
 host_triplet = @host@
 subdir = lib
 subdir = lib
 DIST_COMMON = $(srcdir)/Makefile.am $(srcdir)/Makefile.in
 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)
 DEFAULT_INCLUDES = -I. -I$(srcdir) -I$(top_builddir)
 COMPILE = $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) \
 COMPILE = $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) \
 	$(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS)
 	$(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) \
 	$(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) \
 	$(AM_CFLAGS) $(CFLAGS)
 	$(AM_CFLAGS) $(CFLAGS)
 CCLD = $(CC)
 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 $@
 	$(AM_LDFLAGS) $(LDFLAGS) -o $@
 SOURCES = $(libhadoop_la_SOURCES)
 SOURCES = $(libhadoop_la_SOURCES)
 DIST_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@
 # @configure_input@
 
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
 # 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
 # This Makefile.in is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
 # 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.
 #    assumed to be in $(HADOOP_HOME)/build/native/src/org/apache/hadoop/io/compress/zlib.
 #
 #
 
 
+SOURCES = $(libnativezlib_la_SOURCES)
+
 srcdir = @srcdir@
 srcdir = @srcdir@
 top_srcdir = @top_srcdir@
 top_srcdir = @top_srcdir@
 VPATH = @srcdir@
 VPATH = @srcdir@
@@ -70,7 +72,6 @@ POST_INSTALL = :
 NORMAL_UNINSTALL = :
 NORMAL_UNINSTALL = :
 PRE_UNINSTALL = :
 PRE_UNINSTALL = :
 POST_UNINSTALL = :
 POST_UNINSTALL = :
-build_triplet = @build@
 host_triplet = @host@
 host_triplet = @host@
 subdir = src/org/apache/hadoop/io/compress/zlib
 subdir = src/org/apache/hadoop/io/compress/zlib
 DIST_COMMON = $(srcdir)/Makefile.am $(srcdir)/Makefile.in
 DIST_COMMON = $(srcdir)/Makefile.am $(srcdir)/Makefile.in
@@ -91,11 +92,11 @@ depcomp = $(SHELL) $(top_srcdir)/config/depcomp
 am__depfiles_maybe = depfiles
 am__depfiles_maybe = depfiles
 COMPILE = $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) \
 COMPILE = $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) \
 	$(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS)
 	$(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) \
 	$(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) \
 	$(AM_CFLAGS) $(CFLAGS)
 	$(AM_CFLAGS) $(CFLAGS)
 CCLD = $(CC)
 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 $@
 	$(AM_LDFLAGS) $(LDFLAGS) -o $@
 SOURCES = $(libnativezlib_la_SOURCES)
 SOURCES = $(libnativezlib_la_SOURCES)
 DIST_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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 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.compress.GzipCodec;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.*;
@@ -46,12 +45,12 @@ public class FileBench extends Configured implements Tool {
     System.out.println(
     System.out.println(
 "Usage: Task list:           -[no]r -[no]w\n" +
 "Usage: Task list:           -[no]r -[no]w\n" +
 "       Format:              -[no]seq -[no]txt\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" +
 "       CompressionType:     -[no]blk -[no]rec\n" +
 "       Required:            -dir <working dir>\n" +
 "       Required:            -dir <working dir>\n" +
 "All valid combinations are implicitly enabled, unless an option is enabled\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" +
 "Note that CompressionType params only apply to SequenceFiles\n\n" +
 "Useful options to set:\n" +
 "Useful options to set:\n" +
 "-D fs.default.name=\"file:///\" \\\n" +
 "-D fs.default.name=\"file:///\" \\\n" +
@@ -243,7 +242,7 @@ public class FileBench extends Configured implements Tool {
 
 
   // overwrought argument processing and wordlist follow
   // overwrought argument processing and wordlist follow
   enum CCodec {
   enum CCodec {
-    lzo(LzoCodec.class, ".lzo"), zip(GzipCodec.class, ".gz"), pln(null, "");
+    zip(GzipCodec.class, ".gz"), pln(null, "");
 
 
     Class<? extends CompressionCodec> inf;
     Class<? extends CompressionCodec> inf;
     String ext;
     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.SequenceFile.CompressionType;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.io.compress.LzoCodec;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
 
 
@@ -48,22 +47,6 @@ public class TestSequenceFile extends TestCase {
     LOG.info("Successfully tested SequenceFile with DefaultCodec");
     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 {
   public void compressedSeqFileTest(CompressionCodec codec) throws Exception {
     int count = 1024 * 10;
     int count = 1024 * 10;
     int megabytes = 1;
     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");
     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 {    
   public void testBZip2Codec() throws IOException {    
       codecTest(conf, seed, count, "org.apache.hadoop.io.compress.BZip2Codec");    
       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;
-  }
-
-}

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