Sorry about using attachment. The code is below for the reference.
(I will also file a jira as you suggesting)

package codectest;

import com.hadoop.compression.lzo.LzoCodec;
import java.io.IOException;
import java.util.Formatter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.io.MapFile;
import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.io.compress.SnappyCodec;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;

public class MapFileCodecTest implements Tool {
    private Configuration conf = new Configuration();

    private void createMapFile(Configuration conf, FileSystem fs, String path,
            CompressionCodec codec, CompressionType type, int records)
throws IOException {
        MapFile.Writer writer = new MapFile.Writer(conf, fs, path,
Text.class, Text.class,
                type, codec, null);
        Text key = new Text();
        for (int j = 0; j < records; j++) {
            StringBuilder sb = new StringBuilder();
            Formatter formatter = new Formatter(sb);
            formatter.format("%03d", j);
            key.set(sb.toString());
            writer.append(key, key);
        }
        writer.close();
    }

    private void testCodec(Configuration conf, Class<? extends
CompressionCodec> clazz,
            CompressionType type, int records) throws IOException {
        FileSystem fs = FileSystem.getLocal(conf);
        try {
            System.out.println("Creating MapFiles with " + records  +
                    " records using codec " + clazz.getSimpleName());
            String path = clazz.getSimpleName() + records;
            createMapFile(conf, fs, path, clazz.newInstance(), type, records);
            MapFile.Reader reader = new MapFile.Reader(fs, path, conf);
            Text key1 = new Text("002");
            if (reader.get(key1, new Text()) != null) {
                System.out.println("1st key found");
            }
            Text key2 = new Text("004");
            if (reader.get(key2, new Text()) != null) {
                System.out.println("2nd key found");
            }
        } catch (Throwable ex) {
            ex.printStackTrace();
        }
    }

    @Override
    public int run(String[] strings) throws Exception {
        System.out.println("Using native library " +
System.getProperty("java.library.path"));

        testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 100);
        testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 100);
        testCodec(conf, LzoCodec.class, CompressionType.RECORD, 100);

        testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 10);
        testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 10);
        testCodec(conf, LzoCodec.class, CompressionType.RECORD, 10);

        testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 100);
        testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 100);
        testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 100);

        testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 10);
        testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 10);
        testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 10);
        return 0;
    }

    @Override
    public void setConf(Configuration c) {
        this.conf = c;
    }

    @Override
    public Configuration getConf() {
        return conf;
    }

    public static void main(String[] args) throws Exception {
        ToolRunner.run(new MapFileCodecTest(), args);
    }

}


On 5/21/12, Todd Lipcon <t...@cloudera.com> wrote:
> Hi Jason,
>
> Sounds like a bug. Unfortunately the mailing list strips attachments.
>
> Can you file a jira in the HADOOP project, and attach your test case there?
>
> Thanks
> Todd
>
> On Mon, May 21, 2012 at 3:57 PM, Jason B <urg...@gmail.com> wrote:
>> I am using Cloudera distribution cdh3u1.
>>
>> When trying to check native codecs for better decompression
>> performance such as Snappy or LZO, I ran into issues with random
>> access using MapFile.Reader.get(key, value) method.
>> First call of MapFile.Reader.get() works but a second call fails.
>>
>> Also  I am getting different exceptions depending on number of entries
>> in a map file.
>> With LzoCodec and 10 record file, jvm gets aborted.
>>
>> At the same time the DefaultCodec works fine for all cases, as well as
>> record compression for the native codecs.
>>
>> I created a simple test program (attached) that creates map files
>> locally with sizes of 10 and 100 records for three codecs: Default,
>> Snappy, and LZO.
>> (The test requires corresponding native library available)
>>
>> The summary of problems are given below:
>>
>> Map Size: 100
>> Compression: RECORD
>> ==================
>> DefaultCodec:  OK
>> SnappyCodec: OK
>> LzoCodec: OK
>>
>> Map Size: 10
>> Compression: RECORD
>> ==================
>> DefaultCodec:  OK
>> SnappyCodec: OK
>> LzoCodec: OK
>>
>> Map Size: 100
>> Compression: BLOCK
>> ================
>> DefaultCodec:  OK
>>
>> SnappyCodec: java.io.EOFException  at
>> org.apache.hadoop.io.compress.BlockDecompressorStream.getCompressedData(BlockDecompressorStream.java:114)
>>
>> LzoCodec: java.io.EOFException at
>> org.apache.hadoop.io.compress.BlockDecompressorStream.getCompressedData(BlockDecompressorStream.java:114)
>>
>> Map Size: 10
>> Compression: BLOCK
>> ==================
>> DefaultCodec:  OK
>>
>> SnappyCodec: java.lang.NoClassDefFoundError: Ljava/lang/InternalError
>> at
>> org.apache.hadoop.io.compress.snappy.SnappyDecompressor.decompressBytesDirect(Native
>> Method)
>>
>> LzoCodec:
>> #
>> # A fatal error has been detected by the Java Runtime Environment:
>> #
>> #  SIGSEGV (0xb) at pc=0x00002b068ffcbc00, pid=6385, tid=47304763508496
>> #
>> # JRE version: 6.0_21-b07
>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (17.0-b17 mixed mode
>> linux-amd64 )
>> # Problematic frame:
>> # C  [liblzo2.so.2+0x13c00]  lzo1x_decompress+0x1a0
>> #
>> # An error report file with more information is saved as:
>> # /hadoop/user/yurgis/testapp/hs_err_pid6385.log
>> #
>> # If you would like to submit a bug report, please visit:
>> #   http://java.sun.com/webapps/bugreport/crash.jsp
>> # The crash happened outside the Java Virtual Machine in native code.
>> # See problematic frame for where to report the bug.
>> #
>
>
>
> --
> Todd Lipcon
> Software Engineer, Cloudera
>

Reply via email to